You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2020/11/23 11:35:49 UTC

[GitHub] [nifi-minifi-cpp] hunyadi-dev opened a new pull request #940: MINIFICPP-1389 - Implement ConsumeKafka

hunyadi-dev opened a new pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940


   Notes: 
   1. This Pull request depends on having the librdkafka version updated (#924).
   1. The testing performed so far was run manually only. by having a kafka broker running locally and executing the ConsumeKafkaTest test-cases. This is planned to be automatized in a separate PR by having both the broker and the tests run in a docker container.
   1. Security protocols for kafka are not implemented yet, it will be added in yet another PR.
   1. Polling in batches are not currently handled the same as how NiFi would, I have an open question for Edenhill about how the non-legacy C API is expected to be able to properly batch requests without committing back offsets inbetween poll calls.
   
   This is quite a sizeable PR, upon reviewing, I recommend first going through ConsumeKafkaTests.cpp and comparing the tests with the [requirements listed here](https://nifi.apache.org/docs/nifi-docs/components/nifi-docs/components/org.apache.nifi/nifi-kafka-2-0-nar/1.9.0/org.apache.nifi.processors.kafka.pubsub.ConsumeKafka_2_0/index.html).
   
   `Max Uncommitted Time` and `Communications Timeout` was replaced with `MaxPollTime` and `Session Timeout` options due to API differences. There is also a new property added on how to handle multiple headers of the same keys, called `DuplicateHeaderHandling`.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r577701277



##########
File path: extensions/librdkafka/ConsumeKafka.h
##########
@@ -0,0 +1,181 @@
+/**
+ * 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 <memory>
+#include <string>
+#include <utility>
+#include <vector>
+
+#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 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()) :

Review comment:
       Changed it for this function, and added a Jira for the search-and-replace task:
   https://issues.apache.org/jira/browse/MINIFICPP-1502




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r528647463



##########
File path: libminifi/src/utils/StringUtils.cpp
##########
@@ -35,7 +35,11 @@ std::string StringUtils::trim(std::string s) {
   return trimRight(trimLeft(s));
 }
 
-std::vector<std::string> StringUtils::split(const std::string &str, const std::string &delimiter) {

Review comment:
       Line 43 here could have caused bad memory access in case an empty delimiter. This was a potential SPoF in the `ExpressionLanguage`'s `allDelineatedValues`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r560856528



##########
File path: extensions/librdkafka/tests/ConsumeKafkaTests.cpp
##########
@@ -0,0 +1,590 @@
+
+/**
+ * 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 "../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(), "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 = "Failed to create Kafka producer" + std::string{ errstr.data() };
+      throw std::runtime_error(error_msg);
+    }
+
+    // The last argument is a config here, but it is already owned by the producer. 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> CANCELLED_TRANSACTION        { 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) {
+    reInitialize();
+
+    // Consumer chain
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+    std::shared_ptr<core::Processor> extract_text  = plan_->addProcessor("ExtractText", "extract_text", {success}, false);
+
+    // Set up connections
+    plan_->addConnection(consume_kafka, success, extract_text);
+    extract_text->setAutoTerminatedRelationships({success});
+
+    const auto bool_to_string = [] (const bool b) -> std::string { return b ? "true" : "false"; };
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::SecurityProtocol.getName(), security_protocol);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), topic_names);
+
+    optional_set_property(consume_kafka, ConsumeKafka::TopicNameFormat.getName(), topic_name_format);
+    optional_set_property(consume_kafka, ConsumeKafka::HonorTransactions.getName(), honor_transactions | utils::map(bool_to_string));
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+    optional_set_property(consume_kafka, ConsumeKafka::OffsetReset.getName(), offset_reset);
+    optional_set_property(consume_kafka, ConsumeKafka::KeyAttributeEncoding.getName(), key_attribute_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageDemarcator.getName(), message_demarcator);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageHeaderEncoding.getName(), message_header_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::HeadersToAddAsAttributes.getName(), headers_to_add_as_attributes);
+    optional_set_property(consume_kafka, ConsumeKafka::DuplicateHeaderHandling.getName(), duplicate_header_handling);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    plan_->setProperty(extract_text, ExtractText::Attribute.getName(), ATTRIBUTE_FOR_CAPTURING_CONTENT);
+
+    if (!expect_config_valid) {
+      REQUIRE_THROWS(plan_->scheduleProcessor(consume_kafka));
+      return;
+    } else {
+      plan_->scheduleProcessors();
+    }
+
+    std::unique_ptr<rd_kafka_conf_t, utils::rd_kafka_conf_deleter> conf_;
+    std::unique_ptr<rd_kafka_t, utils::rd_kafka_consumer_deleter> consumer_;
+
+    const bool is_transactional = std::count(transaction_events.cbegin(), transaction_events.cend(), KafkaTestProducer::PublishEvent::TRANSACTION_START);
+    const bool transactions_committed = transaction_events.back() == KafkaTestProducer::PublishEvent::TRANSACTION_COMMIT;
+
+    KafkaTestProducer producer(kafka_brokers, PRODUCER_TOPIC, is_transactional);
+    producer.publish_messages_to_topic(messages_on_topic, TEST_MESSAGE_KEY, transaction_events, message_headers, message_header_encoding);
+
+
+    std::vector<std::shared_ptr<core::FlowFile>> flow_files_produced;
+    for (std::size_t num_expected_messages_processed = 0; num_expected_messages_processed < messages_on_topic.size(); num_expected_messages_processed += std::stoi(max_poll_records.value_or("1"))) {
+      plan_->increment_location();
+      if ((honor_transactions && false == honor_transactions.value()) || (is_transactional && !transactions_committed)) {
+        INFO("Non-committed messages received.");
+        REQUIRE(false == plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+        return;
+      }
+      {
+        SCOPED_INFO("ConsumeKafka timed out when waiting to receive the message published to the kafka broker.");
+        REQUIRE(plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+      }
+      std::size_t num_flow_files_produced = plan_->getNumFlowFileProducedByCurrentProcessor();
+      plan_->increment_location();
+      for (std::size_t times_extract_text_run = 0; times_extract_text_run < num_flow_files_produced; ++times_extract_text_run) {
+        plan_->runCurrentProcessor();  // ExtractText
+        std::shared_ptr<core::FlowFile> flow_file = plan_->getFlowFileProducedByCurrentProcessor();
+        for (const auto& exp_header : expect_header_attributes) {
+          SCOPED_INFO("ConsumeKafka did not produce the expected flowfile attribute from message header: " << exp_header.first << ".");
+          const auto header_attr_opt = flow_file->getAttribute(exp_header.first);
+          REQUIRE(header_attr_opt);
+          REQUIRE(exp_header.second == header_attr_opt.value());
+        }
+        {
+          SCOPED_INFO("Message key is missing or incorrect (potential encoding mismatch).");
+          REQUIRE(TEST_MESSAGE_KEY == decode_key(flow_file->getAttribute(ConsumeKafka::KAFKA_MESSAGE_KEY_ATTR).value(), key_attribute_encoding));
+          REQUIRE("1" == flow_file->getAttribute(ConsumeKafka::KAFKA_COUNT_ATTR).value());
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_OFFSET_ATTR));
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_PARTITION_ATTR));
+          REQUIRE(PRODUCER_TOPIC == flow_file->getAttribute(ConsumeKafka::KAFKA_TOPIC_ATTR).value());
+        }
+        flow_files_produced.emplace_back(std::move(flow_file));
+      }
+      plan_->reset_location();
+    }
+
+    const auto contentOrderOfFlowFile = [&] (const std::shared_ptr<core::FlowFile>& lhs, const std::shared_ptr<core::FlowFile>& rhs) {
+      return lhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value() < rhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value();
+    };
+    {
+      SCOPED_INFO("The flowfiles generated by ConsumeKafka are invalid (probably nullptr).");
+      REQUIRE_NOTHROW(std::sort(flow_files_produced.begin(), flow_files_produced.end(), contentOrderOfFlowFile));
+    }
+    std::vector<std::string> sorted_split_messages = sort_and_split_messages(messages_on_topic, message_demarcator);
+    const auto flow_file_content_matches_message = [&] (const std::shared_ptr<core::FlowFile>& flowfile, const std::string message) {
+      return flowfile->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value() == message;
+    };
+
+    logger_->log_debug("************");
+    std::string expected = "Expected: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      expected += sorted_split_messages[i] + ", ";
+    }
+    std::string   actual = "  Actual: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      actual += flow_files_produced[i]->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value() + ", ";
+    }
+    logger_->log_debug("%s", expected.c_str());
+    logger_->log_debug("%s", actual.c_str());
+    logger_->log_debug("************");
+
+    INFO("The messages received by ConsumeKafka do not match those published");
+    REQUIRE(std::equal(flow_files_produced.begin(), flow_files_produced.end(), sorted_split_messages.begin(), flow_file_content_matches_message));
+  }
+};
+
+class ConsumeKafkaContinuousPublishingTest : public ConsumeKafkaTest {
+ public:
+  ConsumeKafkaContinuousPublishingTest() : ConsumeKafkaTest() {}
+  virtual ~ConsumeKafkaContinuousPublishingTest() {
+    logTestController_.reset();
+  }
+
+  void single_consumer_with_continuous_message_producing(
+      const uint64_t msg_periodicity_ms,
+      const std::string& kafka_brokers,
+      const optional<std::string>& group_id,
+      const optional<std::string>& max_poll_records,
+      const optional<std::string>& max_poll_time,
+      const optional<std::string>& session_timeout) {
+    reInitialize();
+
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+
+    plan_->setProperty(consume_kafka, "allow.auto.create.topics", "true", true);  // Seems like the topic tests work without this
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), PRODUCER_TOPIC);
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    consume_kafka->setAutoTerminatedRelationships({success});
+
+    KafkaTestProducer producer("localhost:9092", PRODUCER_TOPIC, /* transactional = */ false);
+
+    std::atomic_bool producer_loop_stop{ false };
+    const auto producer_loop = [&] {
+      std::size_t num_messages_sent = 0;
+      std::this_thread::sleep_for(std::chrono::milliseconds(100));
+      while (!producer_loop_stop) {
+        producer.publish_messages_to_topic({ "Message after " + std::to_string(msg_periodicity_ms * ++num_messages_sent) + " ms"}, TEST_MESSAGE_KEY, { PUBLISH }, {}, {});
+        std::this_thread::sleep_for(std::chrono::milliseconds(msg_periodicity_ms));
+      }
+    };
+
+    plan_->scheduleProcessors();
+
+    const auto get_time_property_ms = [] (const std::string& property_string) {
+      int64_t value;
+      org::apache::nifi::minifi::core::TimeUnit unit;
+      REQUIRE(org::apache::nifi::minifi::core::Property::StringToTime(property_string, value, unit));
+      int64_t value_as_ms;
+      REQUIRE(org::apache::nifi::minifi::core::Property::ConvertTimeUnitToMS(value, unit, value_as_ms));
+      return value_as_ms;
+    };
+
+    std::thread producer_thread(producer_loop);
+    CHECK_NOTHROW(plan_->runNextProcessor());
+    producer_loop_stop = true;
+    producer_thread.join();
+
+    std::size_t num_flow_files_produced = plan_->getNumFlowFileProducedByCurrentProcessor();
+
+    const uint64_t max_poll_time_ms = get_time_property_ms(max_poll_time.value_or(ConsumeKafka::DEFAULT_MAX_POLL_TIME));
+    const uint64_t max_poll_records_value = max_poll_records ? std::stoi(max_poll_records.value()) : ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+    const uint64_t exp_lower_bound = std::min(max_poll_time_ms / msg_periodicity_ms - 2, max_poll_records_value);
+    const uint64_t exp_upper_bound = std::min(max_poll_time_ms / msg_periodicity_ms + 2, max_poll_records_value);
+    logger_->log_debug("Max poll time: %d, Max poll records: %d, Exp. flowfiles produced: (min: %d, max: %d), actual: %d",
+        max_poll_time_ms, max_poll_records_value, exp_lower_bound, exp_upper_bound, num_flow_files_produced);
+
+    REQUIRE(exp_lower_bound <= num_flow_files_produced);
+    REQUIRE(num_flow_files_produced <= exp_upper_bound);
+  }
+};
+
+const std::string ConsumeKafkaTest::TEST_FILE_NAME_POSTFIX{ "target_kafka_message.txt" };
+const std::string ConsumeKafkaTest::TEST_MESSAGE_KEY{ "consume_kafka_test_key" };
+const std::string ConsumeKafkaTest::PRODUCER_TOPIC{ "ConsumeKafkaTest" };
+const std::string ConsumeKafkaTest::ATTRIBUTE_FOR_CAPTURING_CONTENT{ "flowfile_content" };
+const std::chrono::seconds ConsumeKafkaTest::MAX_CONSUMEKAFKA_POLL_TIME_SECONDS{ 5 };
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "ConsumeKafka parses and uses kafka topics.", "[ConsumeKafka][Kafka][Topic]") {
+  auto run_tests = [&] (const std::vector<std::string>& messages_on_topic, const std::string& topic_names, const optional<std::string>& topic_name_format) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", topic_names, topic_name_format, {}, "test_group_id", {}, {}, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({ "Ulysses",              "James Joyce"         }, "ConsumeKafkaTest",         {});
+  run_tests({ "The Great Gatsby",     "F. Scott Fitzgerald" }, "ConsumeKafkaTest",         ConsumeKafka::TOPIC_FORMAT_NAMES);
+  run_tests({ "War and Peace",        "Lev Tolstoy"         }, "a,b,c,ConsumeKafkaTest,d", ConsumeKafka::TOPIC_FORMAT_NAMES);
+  run_tests({ "Nineteen Eighty Four", "George Orwell"       }, "ConsumeKafkaTest",         ConsumeKafka::TOPIC_FORMAT_PATTERNS);
+  run_tests({ "Hamlet",               "William Shakespeare" }, "Cons[emu]*KafkaTest",      ConsumeKafka::TOPIC_FORMAT_PATTERNS);
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Offsets are reset to the latest when a consumer starts with non-processed messages.", "[ConsumeKafka][Kafka][OffsetReset]") {
+  auto run_tests = [&] (
+      const std::vector<std::string>& messages_on_topic,
+      const std::vector<KafkaTestProducer::PublishEvent>& transaction_events) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, transaction_events, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  KafkaTestProducer producer("localhost:9092", PRODUCER_TOPIC, false);
+  producer.publish_messages_to_topic({"Dummy messages", "that should be ignored", "due to offset reset on ConsumeKafka startup"}, TEST_MESSAGE_KEY, {PUBLISH, PUBLISH, PUBLISH}, {}, {});
+  run_tests({"Brave New World",  "Aldous Huxley"}, NON_TRANSACTIONAL_MESSAGES);
+  producer.publish_messages_to_topic({"Dummy messages", "that should be ignored", "due to offset reset on ConsumeKafka startup"}, TEST_MESSAGE_KEY, {PUBLISH, PUBLISH, PUBLISH}, {}, {});
+  run_tests({"Call of the Wild", "Jack London"}, NON_TRANSACTIONAL_MESSAGES);
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Key attribute is encoded according to the \"Key Attribute Encoding\" property.", "[ConsumeKafka][Kafka][KeyAttributeEncoding]") {
+  auto run_tests = [&] (const std::vector<std::string>& messages_on_topic, const optional<std::string>& key_attribute_encoding) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, key_attribute_encoding, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+
+  run_tests({ "The Odyssey",          "Ὅμηρος"                        }, {});
+  run_tests({ "Lolita",               "Владимир Владимирович Набоков" }, "utf-8");
+  run_tests({ "Crime and Punishment", "Фёдор Михайлович Достоевский"  }, "hex");
+  run_tests({ "Paradise Lost",        "John Milton"                   }, "hEX");
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Transactional behaviour is supported.", "[ConsumeKafka][Kafka][Transaction]") {
+  auto run_tests = [&] (const std::vector<std::string>& messages_on_topic, const std::vector<KafkaTestProducer::PublishEvent>& transaction_events, const optional<bool>& honor_transactions) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, transaction_events, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({  "Pride and Prejudice", "Jane Austen"      }, SINGLE_COMMITTED_TRANSACTION, {});
+  run_tests({                 "Dune", "Frank Herbert"    },    TWO_SEPARATE_TRANSACTIONS, {});
+  run_tests({      "The Black Sheep", "Honore De Balzac" },    NON_COMMITTED_TRANSACTION, {});
+  run_tests({     "Gospel of Thomas"                     },        CANCELLED_TRANSACTION, {});
+  run_tests({ "Operation Dark Heart"                     },        CANCELLED_TRANSACTION, true);
+  run_tests({               "Brexit"                     },        CANCELLED_TRANSACTION, false);
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Headers on consumed Kafka messages are extracted into attributes if requested on ConsumeKafka.", "[ConsumeKafka][Kafka][Headers]") {
+  auto run_tests = [&] (
+      const std::vector<std::string>& messages_on_topic,
+      const std::vector<std::pair<std::string, std::string>>& expect_header_attributes,
+      const std::vector<std::pair<std::string, std::string>>& message_headers,
+      const optional<std::string>& headers_to_add_as_attributes,
+      const optional<std::string>& duplicate_header_handling) {
+    single_consumer_with_plain_text_test(true, false, expect_header_attributes, messages_on_topic, NON_TRANSACTIONAL_MESSAGES, message_headers, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, headers_to_add_as_attributes, duplicate_header_handling, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({             "Homeland",   "R. A. Salvatore"},                                      {},             {{{"Contains dark elves"}, {"Yes"}}},         {},                    {});
+  run_tests({             "Magician",  "Raymond E. Feist"},               {{{"Rating"}, {"10/10"}}},                        {{{"Rating"}, {"10/10"}}}, {"Rating"},                    {});
+  run_tests({             "Mistborn", "Brandon Sanderson"},               {{{"Metal"}, {"Copper"}}}, {{{"Metal"}, {"Copper"}}, {{"Metal"}, {"Iron"}}},  {"Metal"},            KEEP_FIRST);
+  run_tests({             "Mistborn", "Brandon Sanderson"},                 {{{"Metal"}, {"Iron"}}}, {{{"Metal"}, {"Copper"}}, {{"Metal"}, {"Iron"}}},  {"Metal"},           KEEP_LATEST);
+  run_tests({             "Mistborn", "Brandon Sanderson"},         {{{"Metal"}, {"Copper, Iron"}}}, {{{"Metal"}, {"Copper"}}, {{"Metal"}, {"Iron"}}},  {"Metal"}, COMMA_SEPARATED_MERGE);
+  run_tests({"The Lord of the Rings",  "J. R. R. Tolkien"}, {{{"Parts"}, {"First, second, third"}}},          {{{"Parts"}, {"First, second, third"}}},  {"Parts"},                    {});
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Messages are separated into multiple flowfiles if the message demarcator is present in the message.", "[ConsumeKafka][Kafka][MessageDemarcator]") {
+  auto run_tests = [&] (
+      const std::vector<std::string>& messages_on_topic,
+      const optional<std::string>& message_demarcator) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, message_demarcator, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({"Barbapapa", "Anette Tison and Talus Taylor"}, "a");
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "The maximum poll records allows ConsumeKafka to combine multiple messages into a single flowfile.", "[ConsumeKafka][Kafka][Batching][MaxPollRecords]") {
+  auto run_tests = [&] (
+      const std::vector<std::string>& messages_on_topic,
+      const std::vector<KafkaTestProducer::PublishEvent>& transaction_events,
+      const optional<std::string>& max_poll_records) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, transaction_events, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, {}, {}, max_poll_records, "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({"The Count of Monte Cristo", "Alexandre Dumas"}, NON_TRANSACTIONAL_MESSAGES, "2");
+
+  const std::vector<std::string> content {
+      "Make const member functions thread safe",
+      "Understand special member function generation",
+      "Use std::unique_ptr for exclusive-ownership resource management",
+      "Use std::shared_ptr for shared-ownership resource management",
+      "Use std::weak_ptr for std::shared_ptr-like pointers that can dangle",
+      "Prefer std::make_unique and std::make_shared to direct use of new",
+      "When using the Pimpl Idiom, define special member functions inthe implementation file",
+      "Understand std::move and std::forward",
+      "Distinguish universal references from rvalue references",
+      "Use std::move on rvalue references, std::forward on universal references",
+      "Avoid overloading on universal references",
+      "Familiarize yourself with alternatives to overloading on universal references",
+      "Understand reference collapsing",
+      "Assume that move operations are not present, not cheap, and not used",
+      "Familiarize yourself with perfect forwarding failure cases",
+      "Avoid default capture modes",
+      "Use init capture to move objects into closures",
+      "Use decltype on auto&& parameters to std::forward them",
+      "Prefer lambdas to std::bind",
+      "Prefer task-based programming to thread-based" };

Review comment:
       Copy paste from the table of contents of Scott Meyer's Effective Modern C++.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r576711179



##########
File path: extensions/librdkafka/ConsumeKafka.h
##########
@@ -0,0 +1,181 @@
+/**
+ * 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 <memory>
+#include <string>
+#include <utility>
+#include <vector>
+
+#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 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()) :

Review comment:
       Both `name` and `uuid` are taken by value, but are only used as const references. Consider passing by const ref.

##########
File path: libminifi/test/TestBase.cpp
##########
@@ -111,28 +109,19 @@ std::shared_ptr<core::Processor> TestPlan::addProcessor(const std::shared_ptr<co
     }
     relationships_.push_back(connection);
   }
-
   std::shared_ptr<core::ProcessorNode> node = std::make_shared<core::ProcessorNode>(processor);
-
   processor_nodes_.push_back(node);
-
   // std::shared_ptr<core::ProcessContext> context = std::make_shared<core::ProcessContext>(node, controller_services_provider_, prov_repo_, flow_repo_, configuration_, content_repo_);
-
   auto contextBuilder = core::ClassLoader::getDefaultClassLoader().instantiate<core::ProcessContextBuilder>("ProcessContextBuilder");
-
   contextBuilder = contextBuilder->withContentRepository(content_repo_)->withFlowFileRepository(flow_repo_)->withProvider(controller_services_provider_.get())->withProvenanceRepository(prov_repo_)->withConfiguration(configuration_);
-
   auto context = contextBuilder->build(node);
-
   processor_contexts_.push_back(context);
-
   processor_queue_.push_back(processor);
-
   return processor;
 }
 
 std::shared_ptr<core::Processor> TestPlan::addProcessor(const std::string &processor_name, const utils::Identifier& uuid, const std::string &name,
-                                                        const std::initializer_list<core::Relationship>& relationships, bool linkToPrevious) {
+  const std::initializer_list<core::Relationship>& relationships, bool linkToPrevious) {

Review comment:
       Continuation indentation should be 4 spaces

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,578 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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 comma separated list to match against all message headers. Any message header whose name matches an item from the list will be added to the FlowFile "
+      "as an Attribute. If not specified, no Header values will be added as FlowFile attributes. The behaviour on when multiple headers of the same name are present is set using "
+      "the DuplicateHeaderHandling attribute.")
+  ->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 */) {
+  gsl_Expects(context);
+  // 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);
+}
+
+namespace {
+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");
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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:");

Review comment:
       There is a colon here, but not after "assigned" above. Why?

##########
File path: libminifi/test/TestBase.cpp
##########
@@ -150,7 +139,7 @@ std::shared_ptr<core::Processor> TestPlan::addProcessor(const std::string &proce
 }
 
 std::shared_ptr<core::Processor> TestPlan::addProcessor(const std::string &processor_name, const std::string &name, const std::initializer_list<core::Relationship>& relationships,
-                                                        bool linkToPrevious) {
+  bool linkToPrevious) {

Review comment:
       Continuation indentation should be 4 spaces

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,578 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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 comma separated list to match against all message headers. Any message header whose name matches an item from the list will be added to the FlowFile "
+      "as an Attribute. If not specified, no Header values will be added as FlowFile attributes. The behaviour on when multiple headers of the same name are present is set using "
+      "the DuplicateHeaderHandling attribute.")
+  ->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 */) {
+  gsl_Expects(context);
+  // 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);
+}
+
+namespace {
+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");
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Failed to create Kafka consumer " + error_msg);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));
+  }
+  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::steady_clock::now();
+  auto elapsed = std::chrono::steady_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());
+    std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>
+      message { rd_kafka_consumer_poll(consumer_.get(), std::chrono::duration_cast<std::chrono::milliseconds>(max_poll_time_milliseconds_ - elapsed).count()), utils::rd_kafka_message_deleter() };
+    if (!message) {
+      break;
+    }
+    if (RD_KAFKA_RESP_ERR_NO_ERROR != message->err) {
+      logger_->log_error("Received message with error %d: %s", message->err, rd_kafka_err2str(message->err));
+      break;
+    }
+    utils::print_kafka_message(message.get(), *logger_);
+    messages.emplace_back(std::move(message));
+    elapsed = std::chrono::steady_clock::now() - start;
+  }
+  return messages;
+}
+
+utils::KafkaEncoding ConsumeKafka::key_attr_encoding_attr_to_enum() const {
+  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() const {
+  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, "\"Message Header Encoding\" property not recognized.");
+}
+
+std::string ConsumeKafka::resolve_duplicate_headers(const std::vector<std::string>& matching_headers) const {
+  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) const {
+  // 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 < 200) {
+      logger_->log_debug("%.*s", static_cast<int>(size), value);
+    } else {
+      logger_->log_debug("%.*s...", 200, 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) const {
+  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_) {
+      ret = stream->write(data_,  gsl::narrow<int>(dataSize_));
+    }
+    return ret;
+  }
+};

Review comment:
       This should be moved to another scope to avoid generating unnecessary external symbols. I would preferably move it to the relevant function scope, but an anonymous namespace is another alternative.

##########
File path: libminifi/test/unit/StringUtilsTests.cpp
##########
@@ -50,6 +50,16 @@ TEST_CASE("TestStringUtils::split4", "[test split classname]") {
   REQUIRE(expected == StringUtils::split(org::apache::nifi::minifi::core::getClassName<org::apache::nifi::minifi::utils::StringUtils>(), "::"));
 }
 
+TEST_CASE("TestStringUtils::split5", "[test split delimiter not specified]") {

Review comment:
       Now this only tests with empty delimiter. Please add a test for when the delimiter is not specified.

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,578 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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 comma separated list to match against all message headers. Any message header whose name matches an item from the list will be added to the FlowFile "
+      "as an Attribute. If not specified, no Header values will be added as FlowFile attributes. The behaviour on when multiple headers of the same name are present is set using "
+      "the DuplicateHeaderHandling attribute.")
+  ->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 */) {
+  gsl_Expects(context);
+  // 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);
+}
+
+namespace {
+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");
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Failed to create Kafka consumer " + error_msg);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));
+  }
+  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::steady_clock::now();
+  auto elapsed = std::chrono::steady_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());
+    std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>
+      message { rd_kafka_consumer_poll(consumer_.get(), std::chrono::duration_cast<std::chrono::milliseconds>(max_poll_time_milliseconds_ - elapsed).count()), utils::rd_kafka_message_deleter() };
+    if (!message) {
+      break;
+    }
+    if (RD_KAFKA_RESP_ERR_NO_ERROR != message->err) {
+      logger_->log_error("Received message with error %d: %s", message->err, rd_kafka_err2str(message->err));
+      break;
+    }
+    utils::print_kafka_message(message.get(), *logger_);
+    messages.emplace_back(std::move(message));
+    elapsed = std::chrono::steady_clock::now() - start;
+  }
+  return messages;
+}
+
+utils::KafkaEncoding ConsumeKafka::key_attr_encoding_attr_to_enum() const {
+  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() const {
+  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, "\"Message Header Encoding\" property not recognized.");
+}
+
+std::string ConsumeKafka::resolve_duplicate_headers(const std::vector<std::string>& matching_headers) const {
+  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) const {
+  // 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 < 200) {
+      logger_->log_debug("%.*s", static_cast<int>(size), value);
+    } else {
+      logger_->log_debug("%.*s...", 200, 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) const {
+  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_) {
+      ret = stream->write(data_,  gsl::narrow<int>(dataSize_));
+    }
+    return ret;
+  }
+};
+
+void ConsumeKafka::add_kafka_attributes_to_flowfile(std::shared_ptr<FlowFileRecord>& flow_file, const rd_kafka_message_t* message) const {
+  // We do not currently support batching messages into a single flowfile

Review comment:
       I think our flow files are pretty lightweight compared to NiFi, so we don't need flow file batching. But we'll see if this becomes an issue in practice.

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,578 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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 comma separated list to match against all message headers. Any message header whose name matches an item from the list will be added to the FlowFile "
+      "as an Attribute. If not specified, no Header values will be added as FlowFile attributes. The behaviour on when multiple headers of the same name are present is set using "
+      "the DuplicateHeaderHandling attribute.")
+  ->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 */) {
+  gsl_Expects(context);
+  // 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);
+}
+
+namespace {
+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");
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Failed to create Kafka consumer " + error_msg);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));
+  }
+  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::steady_clock::now();
+  auto elapsed = std::chrono::steady_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());
+    std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>
+      message { rd_kafka_consumer_poll(consumer_.get(), std::chrono::duration_cast<std::chrono::milliseconds>(max_poll_time_milliseconds_ - elapsed).count()), utils::rd_kafka_message_deleter() };
+    if (!message) {
+      break;
+    }
+    if (RD_KAFKA_RESP_ERR_NO_ERROR != message->err) {
+      logger_->log_error("Received message with error %d: %s", message->err, rd_kafka_err2str(message->err));
+      break;
+    }
+    utils::print_kafka_message(message.get(), *logger_);
+    messages.emplace_back(std::move(message));
+    elapsed = std::chrono::steady_clock::now() - start;
+  }
+  return messages;
+}
+
+utils::KafkaEncoding ConsumeKafka::key_attr_encoding_attr_to_enum() const {
+  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() const {
+  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, "\"Message Header Encoding\" property not recognized.");
+}
+
+std::string ConsumeKafka::resolve_duplicate_headers(const std::vector<std::string>& matching_headers) const {
+  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) const {
+  // 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 < 200) {
+      logger_->log_debug("%.*s", static_cast<int>(size), value);
+    } else {
+      logger_->log_debug("%.*s...", 200, 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) const {
+  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_) {
+      ret = stream->write(data_,  gsl::narrow<int>(dataSize_));
+    }
+    return ret;
+  }
+};
+
+void ConsumeKafka::add_kafka_attributes_to_flowfile(std::shared_ptr<FlowFileRecord>& flow_file, const rd_kafka_message_t* message) const {
+  // 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, 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));
+}
+
+utils::optional<std::vector<std::shared_ptr<FlowFileRecord>>> ConsumeKafka::transform_pending_messages_into_flowfiles(core::ProcessSession* session) const {

Review comment:
       This should probably take a reference instead of a pointer, to avoid null pointer problems. Please check all functions taking a raw pointer, except for onTrigger/onSchedule, which are constrained by the base class interface.
   
   In general, we should make sure that whenever we're working with a nullable pointer, we don't dereference it without checking in the same scope. When we don't need nullability, references (for passing) or gsl::not_null (for storage) can be used instead.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r540190615



##########
File path: libminifi/src/utils/StringUtils.cpp
##########
@@ -59,13 +63,21 @@ std::vector<std::string> StringUtils::split(const std::string &str, const std::s
       break;
     }
     auto next = std::find_if(curr, end, is_func);
-    result.push_back(std::string(curr, next));
+    result.push_back(transformation(std::string(curr, next)));

Review comment:
       Done.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] fgerlits commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
fgerlits commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553417399



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  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() const {
+  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() const {
+  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) const {
+  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) const {
+  // 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()) {

Review comment:
       These are headers, not messages.  Is a single header larger than 2 GB a valid use case?
   
   Also, logging a 2 GB long string is not a good idea.  It would probably be better to log the first 100 or 200 characters, followed by "..." if there are more.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553877279



##########
File path: libminifi/include/core/TypedValues.h
##########
@@ -106,6 +106,8 @@ class TimePeriodValue : public TransformableValue, public state::response::UInt6
   static bool StringToTime(std::string input, uint64_t &output, TimeUnit &timeunit) {
     return utils::internal::StringToTime(input, output, timeunit);
   }
+
+  TimePeriodValue& operator=(const TimePeriodValue& other) = default;

Review comment:
       I don't remember anymore :) Removed.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553241026



##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);
+  }
+}
+
+void print_topics_list(std::shared_ptr<logging::Logger> logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger->log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld]",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, const std::shared_ptr<logging::Logger>& logger) {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    const std::string error_msg = "ConsumeKafka: received error message from broker. Librdkafka error msg: " + std::string(rd_kafka_err2str(rkmessage->err));
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, error_msg);

Review comment:
       ~I recommend passing a non-const rvalue to the `Exception` constructor to avoid a long string copy.~
   
   edit: I realized that due to noexcept copy requirements of exceptions, the message is copied in any case, so ignore this.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553850276



##########
File path: libminifi/include/utils/ProcessorConfigUtils.h
##########
@@ -0,0 +1,80 @@
+/**
+ * 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) {
+  std::string value;
+  if (!context->getProperty(property_name, value)) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property missing or invalid");
+  }
+  return value;
+}
+
+std::vector<std::string> listFromCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  std::string property_string;
+  context->getProperty(property_name, property_string);
+  return utils::StringUtils::splitAndTrim(property_string, ",");
+}
+
+std::vector<std::string> listFromRequiredCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  return utils::StringUtils::splitAndTrim(getRequiredPropertyOrThrow(context, property_name), ",");
+}
+
+bool parseBooleanPropertyOrThrow(core::ProcessContext* context, const std::string& property_name) {
+  bool value;
+  std::string value_str = getRequiredPropertyOrThrow(context, property_name);
+  utils::optional<bool> maybe_value = utils::StringUtils::toBool(value_str);
+  if (!maybe_value) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property is invalid: value is " + value_str);

Review comment:
       I'm fine with `std::runtime_error` or `std::invalid_argument` as well, but I don't want miscategorized exceptions.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r551920589



##########
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();
+  }
+};

Review comment:
       Moved there as requested.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r560987064



##########
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;
+    // }
+    rd_kafka_destroy(ptr);
+  }
+};
+
+struct rd_kafka_consumer_deleter {
+  void operator()(rd_kafka_t* ptr) const noexcept {
+    rd_kafka_consumer_close(ptr);
+    rd_kafka_destroy(ptr);
+  }
+};
+
+struct rd_kafka_topic_partition_list_deleter {
+  void operator()(rd_kafka_topic_partition_list_t* ptr) const noexcept { rd_kafka_topic_partition_list_destroy(ptr); }
+};
+
+struct rd_kafka_topic_conf_deleter {
+  void operator()(rd_kafka_topic_conf_t* ptr) const noexcept { rd_kafka_topic_conf_destroy(ptr); }
+};
+struct rd_kafka_topic_deleter {
+  void operator()(rd_kafka_topic_t* ptr) const noexcept { rd_kafka_topic_destroy(ptr); }
+};
+
+struct rd_kafka_message_deleter {
+  void operator()(rd_kafka_message_t* ptr) const noexcept { rd_kafka_message_destroy(ptr); }
+};
+
+struct rd_kafka_headers_deleter {
+  void operator()(rd_kafka_headers_t* ptr) const noexcept { rd_kafka_headers_destroy(ptr); }
+};
+
+template <typename T>
+void kafka_headers_for_each(const rd_kafka_headers_t* headers, T key_value_handle) {
+  const char *key;  // Null terminated, not to be freed
+  const void *value;
+  std::size_t size;
+  for (std::size_t i = 0; RD_KAFKA_RESP_ERR_NO_ERROR == rd_kafka_header_get_all(headers, i, &key, &value, &size); ++i) {
+    key_value_handle(std::string(key), std::string(static_cast<const char*>(value), size));

Review comment:
       In that case pass a span down. The point is that the copy is not necessary because the usage is fully enclosed in the loop body i.e. the lifetime of `*value`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev edited a comment on pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev edited a comment on pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#issuecomment-781153230


   > If you want to keep the librdkafka types as they are to more closely resemble their usage in C, I'm fine with that, otherwise here's a list of the remaining pointers parameters in the PR that are used in the body without a null check.
   
   I liked that the current interface is easy to match to the rdkafka api, but it is only a mild annoyance having to revisit this PR over and over, so I don't mind changing it.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553890911



##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);
+  }
+}
+
+void print_topics_list(std::shared_ptr<logging::Logger> logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger->log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld]",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, const std::shared_ptr<logging::Logger>& logger) {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    const std::string error_msg = "ConsumeKafka: received error message from broker. Librdkafka error msg: " + std::string(rd_kafka_err2str(rkmessage->err));
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, error_msg);
+  }
+  std::string topicName = rd_kafka_topic_name(rkmessage->rkt);
+  std::string message(reinterpret_cast<char*>(rkmessage->payload), rkmessage->len);
+  const char* key = reinterpret_cast<const char*>(rkmessage->key);
+  const std::size_t key_len = rkmessage->key_len;
+  rd_kafka_timestamp_type_t tstype;
+  int64_t timestamp;
+  timestamp = rd_kafka_message_timestamp(rkmessage, &tstype);
+  const char *tsname = "?";
+  if (tstype != RD_KAFKA_TIMESTAMP_NOT_AVAILABLE) {
+    if (tstype == RD_KAFKA_TIMESTAMP_CREATE_TIME) {
+      tsname = "create time";
+    } else if (tstype == RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME) {
+      tsname = "log append time";
+    }
+  }
+  const int64_t seconds_since_timestamp = timestamp ? static_cast<int64_t>(time(NULL)) - static_cast<int64_t>(timestamp / 1000) : 0;
+
+  std::string headers_as_string;
+  rd_kafka_headers_t* hdrs;
+  const rd_kafka_resp_err_t get_header_response = rd_kafka_message_headers(rkmessage, &hdrs);
+  if (RD_KAFKA_RESP_ERR_NO_ERROR == get_header_response) {
+    std::vector<std::string> header_list;
+    kafka_headers_for_each(hdrs, [&] (const std::string& key, const std::string& val) { header_list.emplace_back(key + ": " + val); });
+    headers_as_string = StringUtils::join(", ", header_list);
+  } else if (RD_KAFKA_RESP_ERR__NOENT != 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::string message_as_string;
+  message_as_string += "[Topic](" + topicName + "), ";
+  message_as_string += "[Key](" + (key != nullptr ? std::string(key, key_len) : std::string("[None]")) + "), ";
+  message_as_string += "[Offset](" +  std::to_string(rkmessage->offset) + "), ";
+  message_as_string += "[Message Length](" + std::to_string(rkmessage->len) + "), ";
+  message_as_string += "[Timestamp](" + std::string(tsname) + " " + std::to_string(timestamp) + " (" + std::to_string(seconds_since_timestamp) + " s ago)), ";
+  message_as_string += "[Headers](";
+  message_as_string += headers_as_string + "\n";
+  message_as_string += "[Payload](" + message + ")";
+
+  logger -> log_debug("Message: %s", message_as_string.c_str());

Review comment:
       Seems like this was an outdated commit, fixed since then.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553348682



##########
File path: libminifi/include/core/FlowFile.h
##########
@@ -134,7 +135,9 @@ class FlowFile : public CoreComponent, public ReferenceContainer {
    * @param value value to set
    * @return result of finding key
    */
-  bool getAttribute(std::string key, std::string& value) const;
+  bool getAttribute(const std::string& key, std::string& value) const;
+
+  utils::optional<std::reference_wrapper<const std::string>> getAttribute(const std::string& key) const;

Review comment:
       Good observation, this was cherry-picked from #911 hoping that it would be merged much much earlier than this PR, whereas it is still open. I think I will just close that PR and move the change here.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#issuecomment-781153230


   > If you want to keep the librdkafka types as they are to more closely resemble their usage in C, I'm fine with that, otherwise here's a list of the remaining pointers parameters in the PR that are used in the body without a null check.
   
   I liked that the current interface is easy to match to the rdkafka api, but it is only a mild annoyance having to revisit this PR over and over.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553403462



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,569 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));
+  }
+  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::steady_clock::now();
+  auto elapsed = std::chrono::steady_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());

Review comment:
       Removed padding.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r540161207



##########
File path: libminifi/test/unit/StringUtilsTests.cpp
##########
@@ -50,6 +50,16 @@ TEST_CASE("TestStringUtils::split4", "[test split classname]") {
   REQUIRE(expected == StringUtils::split(org::apache::nifi::minifi::core::getClassName<org::apache::nifi::minifi::utils::StringUtils>(), "::"));
 }
 
+TEST_CASE("TestStringUtils::split5", "[test split delimiter not specified]") {

Review comment:
       I see, updated the function for splitting into characters on empty delimiter.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r540113524



##########
File path: libminifi/test/unit/StringUtilsTests.cpp
##########
@@ -50,6 +50,16 @@ TEST_CASE("TestStringUtils::split4", "[test split classname]") {
   REQUIRE(expected == StringUtils::split(org::apache::nifi::minifi::core::getClassName<org::apache::nifi::minifi::utils::StringUtils>(), "::"));
 }
 
+TEST_CASE("TestStringUtils::split5", "[test split delimiter not specified]") {

Review comment:
       Default (no delimiter specified) and empty delimiter are different. Python fails on empty delimiter, ruby splits by characters.
   
   The delimiter can have a default argument of `" "` (whitespace).




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r538356594



##########
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:
       Yes, updated accordingly.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553912946



##########
File path: libminifi/include/core/FlowFile.h
##########
@@ -134,7 +135,9 @@ class FlowFile : public CoreComponent, public ReferenceContainer {
    * @param value value to set
    * @return result of finding key
    */
-  bool getAttribute(std::string key, std::string& value) const;
+  bool getAttribute(const std::string& key, std::string& value) const;
+
+  utils::optional<std::reference_wrapper<const std::string>> getAttribute(const std::string& key) const;

Review comment:
       Updated.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r576144662



##########
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;
+    // }
+    rd_kafka_destroy(ptr);
+  }
+};
+
+struct rd_kafka_consumer_deleter {
+  void operator()(rd_kafka_t* ptr) const noexcept {
+    rd_kafka_consumer_close(ptr);
+    rd_kafka_destroy(ptr);
+  }
+};
+
+struct rd_kafka_topic_partition_list_deleter {
+  void operator()(rd_kafka_topic_partition_list_t* ptr) const noexcept { rd_kafka_topic_partition_list_destroy(ptr); }
+};
+
+struct rd_kafka_topic_conf_deleter {
+  void operator()(rd_kafka_topic_conf_t* ptr) const noexcept { rd_kafka_topic_conf_destroy(ptr); }
+};
+struct rd_kafka_topic_deleter {
+  void operator()(rd_kafka_topic_t* ptr) const noexcept { rd_kafka_topic_destroy(ptr); }
+};
+
+struct rd_kafka_message_deleter {
+  void operator()(rd_kafka_message_t* ptr) const noexcept { rd_kafka_message_destroy(ptr); }
+};
+
+struct rd_kafka_headers_deleter {
+  void operator()(rd_kafka_headers_t* ptr) const noexcept { rd_kafka_headers_destroy(ptr); }
+};
+
+template <typename T>
+void kafka_headers_for_each(const rd_kafka_headers_t* headers, T key_value_handle) {
+  const char *key;  // Null terminated, not to be freed
+  const void *value;
+  std::size_t size;
+  for (std::size_t i = 0; RD_KAFKA_RESP_ERR_NO_ERROR == rd_kafka_header_get_all(headers, i, &key, &value, &size); ++i) {
+    key_value_handle(std::string(key), std::string(static_cast<const char*>(value), size));

Review comment:
       I replaced this with a `gsl::span`, but there is still a copy - now performed by the caller, and the function signature looks less clean.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553912267



##########
File path: extensions/librdkafka/docker_tests/CMakeLists.txt
##########
@@ -0,0 +1,36 @@
+#

Review comment:
       I mean these will sit in docker, but right now they are just something to run manually on minifi with a working local broker. I will soon get to start working on dockerized tests, that PR will probably get rid of these.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553890971



##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);

Review comment:
       Added missing " :".




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
lordgamez commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r534279245



##########
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:
       Sorry, I was referring to processing the incoming flow files for evaluating the expression attribute values, but I see that there are no incoming flow files for this processor so that's not an option. You can ignore this comment.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553237058



##########
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;
+    // }
+    rd_kafka_destroy(ptr);
+  }
+};
+
+struct rd_kafka_consumer_deleter {
+  void operator()(rd_kafka_t* ptr) const noexcept {
+    rd_kafka_consumer_close(ptr);
+    rd_kafka_destroy(ptr);
+  }
+};
+
+struct rd_kafka_topic_partition_list_deleter {
+  void operator()(rd_kafka_topic_partition_list_t* ptr) const noexcept { rd_kafka_topic_partition_list_destroy(ptr); }
+};
+
+struct rd_kafka_topic_conf_deleter {
+  void operator()(rd_kafka_topic_conf_t* ptr) const noexcept { rd_kafka_topic_conf_destroy(ptr); }
+};
+struct rd_kafka_topic_deleter {
+  void operator()(rd_kafka_topic_t* ptr) const noexcept { rd_kafka_topic_destroy(ptr); }
+};
+
+struct rd_kafka_message_deleter {
+  void operator()(rd_kafka_message_t* ptr) const noexcept { rd_kafka_message_destroy(ptr); }
+};
+
+struct rd_kafka_headers_deleter {
+  void operator()(rd_kafka_headers_t* ptr) const noexcept { rd_kafka_headers_destroy(ptr); }
+};
+
+template <typename T>
+void kafka_headers_for_each(const rd_kafka_headers_t* headers, T&& key_value_handle) {
+  const char *key;  // Null terminated, not to be freed
+  const void *value;
+  std::size_t size;
+  for (std::size_t i = 0; RD_KAFKA_RESP_ERR_NO_ERROR == rd_kafka_header_get_all(headers, i, &key, &value, &size); ++i) {
+    std::forward<T>(key_value_handle)(std::string(key), std::string(static_cast<const char*>(value), size));

Review comment:
       Repeated calls to rvalue function objects is dangerous. I suggest to pass `key_value_handle` by value.

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);
+  }
+}
+
+void print_topics_list(std::shared_ptr<logging::Logger> logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger->log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld]",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, const std::shared_ptr<logging::Logger>& logger) {

Review comment:
       These functions should take a `Logger&` instead, because they have no reason to depend on any lifetime or ownership structure of the logger, and extra dependencies add complexity.

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,569 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));
+  }
+  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::steady_clock::now();
+  auto elapsed = std::chrono::steady_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());
+    std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>
+      message { rd_kafka_consumer_poll(consumer_.get(), std::chrono::duration_cast<std::chrono::milliseconds>(max_poll_time_milliseconds_ - elapsed).count()), utils::rd_kafka_message_deleter() };
+    if (!message) {
+      break;
+    }
+    if (RD_KAFKA_RESP_ERR_NO_ERROR != message->err) {
+      logger_->log_error("Received message with error %d: %s", message->err, rd_kafka_err2str(message->err));
+      break;
+    }
+    utils::print_kafka_message(message.get(), logger_);
+    messages.emplace_back(std::move(message));
+    elapsed = std::chrono::steady_clock::now() - start;
+  }
+  return messages;
+}
+
+utils::KafkaEncoding ConsumeKafka::key_attr_encoding_attr_to_enum() const {
+  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() const {
+  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, "\"Message Header Encoding\" property not recognized.");
+}
+
+std::string ConsumeKafka::resolve_duplicate_headers(const std::vector<std::string>& matching_headers) const {
+  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) const {
+  // 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) const {
+  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_) {
+      ret = stream->write(data_,  gsl::narrow<int>(dataSize_));
+    }
+    return ret;
+  }
+};
+
+void ConsumeKafka::add_kafka_attributes_to_flowfile(std::shared_ptr<FlowFileRecord>& flow_file, const rd_kafka_message_t* message) const {
+  // 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, 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));
+}
+
+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) const {
+  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("Message received contains no data.");
+      continue;
+    }

Review comment:
       I think we should create empty flow files for those messages. There might be useful information in the headers or even the existence of such a message might be meaningful.

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);

Review comment:
       We should have some separation of the error message, like `": "`
   ```suggestion
       throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error: " + error_msg);
   ```

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);
+  }
+}
+
+void print_topics_list(std::shared_ptr<logging::Logger> logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger->log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld]",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, const std::shared_ptr<logging::Logger>& logger) {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    const std::string error_msg = "ConsumeKafka: received error message from broker. Librdkafka error msg: " + std::string(rd_kafka_err2str(rkmessage->err));
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, error_msg);

Review comment:
       I recommend passing a non-const rvalue to the `Exception` constructor to avoid a long string copy.
   ```suggestion
       throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, 
           "ConsumeKafka: received error message from broker. Librdkafka error msg: " + std::string(rd_kafka_err2str(rkmessage->err)));
   ```

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);
+  }
+}
+
+void print_topics_list(std::shared_ptr<logging::Logger> logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger->log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld]",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, const std::shared_ptr<logging::Logger>& logger) {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    const std::string error_msg = "ConsumeKafka: received error message from broker. Librdkafka error msg: " + std::string(rd_kafka_err2str(rkmessage->err));
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, error_msg);
+  }
+  std::string topicName = rd_kafka_topic_name(rkmessage->rkt);
+  std::string message(reinterpret_cast<char*>(rkmessage->payload), rkmessage->len);
+  const char* key = reinterpret_cast<const char*>(rkmessage->key);
+  const std::size_t key_len = rkmessage->key_len;
+  rd_kafka_timestamp_type_t tstype;
+  int64_t timestamp;
+  timestamp = rd_kafka_message_timestamp(rkmessage, &tstype);
+  const char *tsname = "?";
+  if (tstype != RD_KAFKA_TIMESTAMP_NOT_AVAILABLE) {
+    if (tstype == RD_KAFKA_TIMESTAMP_CREATE_TIME) {
+      tsname = "create time";
+    } else if (tstype == RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME) {
+      tsname = "log append time";
+    }
+  }
+  const int64_t seconds_since_timestamp = timestamp ? static_cast<int64_t>(time(NULL)) - static_cast<int64_t>(timestamp / 1000) : 0;
+
+  std::string headers_as_string;
+  rd_kafka_headers_t* hdrs;
+  const rd_kafka_resp_err_t get_header_response = rd_kafka_message_headers(rkmessage, &hdrs);
+  if (RD_KAFKA_RESP_ERR_NO_ERROR == get_header_response) {
+    std::vector<std::string> header_list;
+    kafka_headers_for_each(hdrs, [&] (const std::string& key, const std::string& val) { header_list.emplace_back(key + ": " + val); });
+    headers_as_string = StringUtils::join(", ", header_list);
+  } else if (RD_KAFKA_RESP_ERR__NOENT != 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::string message_as_string;
+  message_as_string += "[Topic](" + topicName + "), ";
+  message_as_string += "[Key](" + (key != nullptr ? std::string(key, key_len) : std::string("[None]")) + "), ";
+  message_as_string += "[Offset](" +  std::to_string(rkmessage->offset) + "), ";
+  message_as_string += "[Message Length](" + std::to_string(rkmessage->len) + "), ";
+  message_as_string += "[Timestamp](" + std::string(tsname) + " " + std::to_string(timestamp) + " (" + std::to_string(seconds_since_timestamp) + " s ago)), ";
+  message_as_string += "[Headers](";
+  message_as_string += headers_as_string + "\n";
+  message_as_string += "[Payload](" + message + ")";
+
+  logger -> log_debug("Message: %s", message_as_string.c_str());
+}
+
+std::string get_encoded_string(const std::string& input, KafkaEncoding encoding) {
+  switch (encoding) {
+    case KafkaEncoding::UTF8:
+      return input;
+    case KafkaEncoding::HEX:
+      return StringUtils::to_hex(input, /* uppercase = */ true);
+  }
+  throw std::runtime_error("Invalid encoding selected for encoding.");

Review comment:
       Word repetition makes this error message feel strange. My suggestion, including more details:
   ```suggestion
     throw std::runtime_error(fmt::format("get_encoded_string: invalid encoding ({})", static_cast<int>(encoding)));
   ```

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,562 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);

Review comment:
       The error string contains a format specifier, but it's not used in a formatting function. 
   ```suggestion
       throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Failed to create Kafka consumer " + error_msg);
   ```

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);
+  }
+}
+
+void print_topics_list(std::shared_ptr<logging::Logger> logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger->log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld]",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, const std::shared_ptr<logging::Logger>& logger) {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    const std::string error_msg = "ConsumeKafka: received error message from broker. Librdkafka error msg: " + std::string(rd_kafka_err2str(rkmessage->err));
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, error_msg);
+  }
+  std::string topicName = rd_kafka_topic_name(rkmessage->rkt);
+  std::string message(reinterpret_cast<char*>(rkmessage->payload), rkmessage->len);
+  const char* key = reinterpret_cast<const char*>(rkmessage->key);
+  const std::size_t key_len = rkmessage->key_len;
+  rd_kafka_timestamp_type_t tstype;
+  int64_t timestamp;
+  timestamp = rd_kafka_message_timestamp(rkmessage, &tstype);
+  const char *tsname = "?";
+  if (tstype != RD_KAFKA_TIMESTAMP_NOT_AVAILABLE) {
+    if (tstype == RD_KAFKA_TIMESTAMP_CREATE_TIME) {
+      tsname = "create time";
+    } else if (tstype == RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME) {
+      tsname = "log append time";
+    }
+  }
+  const int64_t seconds_since_timestamp = timestamp ? static_cast<int64_t>(time(NULL)) - static_cast<int64_t>(timestamp / 1000) : 0;
+
+  std::string headers_as_string;
+  rd_kafka_headers_t* hdrs;
+  const rd_kafka_resp_err_t get_header_response = rd_kafka_message_headers(rkmessage, &hdrs);
+  if (RD_KAFKA_RESP_ERR_NO_ERROR == get_header_response) {
+    std::vector<std::string> header_list;
+    kafka_headers_for_each(hdrs, [&] (const std::string& key, const std::string& val) { header_list.emplace_back(key + ": " + val); });
+    headers_as_string = StringUtils::join(", ", header_list);
+  } else if (RD_KAFKA_RESP_ERR__NOENT != 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::string message_as_string;
+  message_as_string += "[Topic](" + topicName + "), ";
+  message_as_string += "[Key](" + (key != nullptr ? std::string(key, key_len) : std::string("[None]")) + "), ";
+  message_as_string += "[Offset](" +  std::to_string(rkmessage->offset) + "), ";
+  message_as_string += "[Message Length](" + std::to_string(rkmessage->len) + "), ";
+  message_as_string += "[Timestamp](" + std::string(tsname) + " " + std::to_string(timestamp) + " (" + std::to_string(seconds_since_timestamp) + " s ago)), ";
+  message_as_string += "[Headers](";
+  message_as_string += headers_as_string + "\n";
+  message_as_string += "[Payload](" + message + ")";
+
+  logger -> log_debug("Message: %s", message_as_string.c_str());

Review comment:
       Extra spaces around `->`

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,569 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {

Review comment:
       Extra spaces around `->`

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,569 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {

Review comment:
       Extra spaces around `->`

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);
+  }
+}
+
+void print_topics_list(std::shared_ptr<logging::Logger> logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger->log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld]",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, const std::shared_ptr<logging::Logger>& logger) {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    const std::string error_msg = "ConsumeKafka: received error message from broker. Librdkafka error msg: " + std::string(rd_kafka_err2str(rkmessage->err));
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, error_msg);
+  }
+  std::string topicName = rd_kafka_topic_name(rkmessage->rkt);
+  std::string message(reinterpret_cast<char*>(rkmessage->payload), rkmessage->len);
+  const char* key = reinterpret_cast<const char*>(rkmessage->key);
+  const std::size_t key_len = rkmessage->key_len;
+  rd_kafka_timestamp_type_t tstype;
+  int64_t timestamp;
+  timestamp = rd_kafka_message_timestamp(rkmessage, &tstype);
+  const char *tsname = "?";
+  if (tstype != RD_KAFKA_TIMESTAMP_NOT_AVAILABLE) {
+    if (tstype == RD_KAFKA_TIMESTAMP_CREATE_TIME) {
+      tsname = "create time";
+    } else if (tstype == RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME) {
+      tsname = "log append time";
+    }
+  }
+  const int64_t seconds_since_timestamp = timestamp ? static_cast<int64_t>(time(NULL)) - static_cast<int64_t>(timestamp / 1000) : 0;

Review comment:
       Is there a reason for printing a relative timestamp instead of the actual absolute one? It takes <= 1 sec of being on the screen (or in the log) for a relative timestamp to become outdated.

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,569 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));

Review comment:
       Instead of repeated string concatenations, I would use `fmt::format` or `StringUtils::join_pack`

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,569 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));
+  }
+  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::steady_clock::now();
+  auto elapsed = std::chrono::steady_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());
+    std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>
+      message { rd_kafka_consumer_poll(consumer_.get(), std::chrono::duration_cast<std::chrono::milliseconds>(max_poll_time_milliseconds_ - elapsed).count()), utils::rd_kafka_message_deleter() };
+    if (!message) {
+      break;
+    }
+    if (RD_KAFKA_RESP_ERR_NO_ERROR != message->err) {
+      logger_->log_error("Received message with error %d: %s", message->err, rd_kafka_err2str(message->err));
+      break;
+    }
+    utils::print_kafka_message(message.get(), logger_);
+    messages.emplace_back(std::move(message));
+    elapsed = std::chrono::steady_clock::now() - start;
+  }
+  return messages;
+}
+
+utils::KafkaEncoding ConsumeKafka::key_attr_encoding_attr_to_enum() const {
+  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() const {
+  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, "\"Message Header Encoding\" property not recognized.");
+}
+
+std::string ConsumeKafka::resolve_duplicate_headers(const std::vector<std::string>& matching_headers) const {
+  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) const {
+  // 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) const {
+  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_) {
+      ret = stream->write(data_,  gsl::narrow<int>(dataSize_));
+    }
+    return ret;
+  }
+};
+
+void ConsumeKafka::add_kafka_attributes_to_flowfile(std::shared_ptr<FlowFileRecord>& flow_file, const rd_kafka_message_t* message) const {
+  // 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, 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));
+}
+
+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) const {
+  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("Message received contains no data.");
+      continue;
+    }
+
+    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 { message_content };
+    if (message_demarcator_.size()) {
+      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) {
+        logger_->log_error("Failed to create flowfile.");
+        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);
+      }
+      add_kafka_attributes_to_flowfile(flow_file, message.get());
+      flow_files_created.emplace_back(std::move(flow_file));
+    }
+  }
+  return flow_files_created;
+}
+
+void ConsumeKafka::onTrigger(core::ProcessContext* /* context */, core::ProcessSession* session) {
+  logger_->log_debug("ConsumeKafka onTrigger");
+
+  std::vector<std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>> messages = poll_kafka_messages();
+
+  std::vector<std::shared_ptr<FlowFileRecord>> flow_files_created = transform_messages_into_flowfiles(messages, session);
+  if (flow_files_created.empty()) {
+    return;
+  }
+  for (const auto& flow_file : flow_files_created) {
+    session->transfer(flow_file, Success);
+  }
+  session->commit();
+  // Commit the offset from the latest message only
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rd_kafka_commit_message(consumer_.get(), messages.back().get(), /* async = */ 0)) {
+    logger_ -> log_error("Committing offsets failed.");

Review comment:
       Extra spaces around `->`

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,569 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));
+  }
+  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::steady_clock::now();
+  auto elapsed = std::chrono::steady_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());
+    std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>
+      message { rd_kafka_consumer_poll(consumer_.get(), std::chrono::duration_cast<std::chrono::milliseconds>(max_poll_time_milliseconds_ - elapsed).count()), utils::rd_kafka_message_deleter() };
+    if (!message) {
+      break;
+    }
+    if (RD_KAFKA_RESP_ERR_NO_ERROR != message->err) {
+      logger_->log_error("Received message with error %d: %s", message->err, rd_kafka_err2str(message->err));
+      break;
+    }
+    utils::print_kafka_message(message.get(), logger_);
+    messages.emplace_back(std::move(message));
+    elapsed = std::chrono::steady_clock::now() - start;
+  }
+  return messages;
+}
+
+utils::KafkaEncoding ConsumeKafka::key_attr_encoding_attr_to_enum() const {
+  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() const {
+  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, "\"Message Header Encoding\" property not recognized.");
+}
+
+std::string ConsumeKafka::resolve_duplicate_headers(const std::vector<std::string>& matching_headers) const {
+  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) const {
+  // 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) const {
+  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_) {
+      ret = stream->write(data_,  gsl::narrow<int>(dataSize_));
+    }
+    return ret;
+  }
+};
+
+void ConsumeKafka::add_kafka_attributes_to_flowfile(std::shared_ptr<FlowFileRecord>& flow_file, const rd_kafka_message_t* message) const {
+  // 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, 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));
+}
+
+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) const {
+  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("Message received contains no data.");
+      continue;
+    }
+
+    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 { message_content };
+    if (message_demarcator_.size()) {
+      split_message = utils::StringUtils::split(message_content, message_demarcator_);
+    }

Review comment:
       I think we can and should avoid these two allocations if the message is to be split anyway.
   ```suggestion
       const auto split_message = [&] {
         if (message_demarcator_.size()) {
           return utils::StringUtils::split(message_content, message_demarcator_);
         } else {
           return std::vector<std::string>{ message_content };
         }
       }();
   ```

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,569 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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.");

Review comment:
       Extra spaces around `->`

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,569 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));
+  }
+  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::steady_clock::now();
+  auto elapsed = std::chrono::steady_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());

Review comment:
       Extra space after `->`




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r567644990



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,579 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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.")

Review comment:
       Reworded the property description here and in `Processors.md`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r578221481



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,570 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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 comma separated list to match against all message headers. Any message header whose name matches an item from the list will be added to the FlowFile "
+      "as an Attribute. If not specified, no Header values will be added as FlowFile attributes. The behaviour on when multiple headers of the same name are present is set using "
+      "the DuplicateHeaderHandling attribute.")
+  ->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 */) {
+  gsl_Expects(context);
+  // 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);
+}
+
+namespace {
+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:");
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Failed to create Kafka consumer " + error_msg);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {

Review comment:
       Updated.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553897459



##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);
+  }
+}
+
+void print_topics_list(std::shared_ptr<logging::Logger> logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger->log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld]",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, const std::shared_ptr<logging::Logger>& logger) {

Review comment:
       Taking a shared pointer instead of an observer pointer/reference makes it impossible to use the function with anything other than a shared pointer, like unique ptr, stack object, manually allocated or member of another object.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r567639924



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,579 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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.")

Review comment:
       The property was taken from the NiFi property description, it is true that we do not do any bundling, Will delete this part of the property description.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r577476694



##########
File path: libminifi/test/TestBase.cpp
##########
@@ -150,7 +139,7 @@ std::shared_ptr<core::Processor> TestPlan::addProcessor(const std::string &proce
 }
 
 std::shared_ptr<core::Processor> TestPlan::addProcessor(const std::string &processor_name, const std::string &name, const std::initializer_list<core::Relationship>& relationships,
-                                                        bool linkToPrevious) {
+  bool linkToPrevious) {

Review comment:
       That is correct, updated.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553796377



##########
File path: libminifi/include/utils/ProcessorConfigUtils.h
##########
@@ -0,0 +1,80 @@
+/**
+ * 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) {
+  std::string value;
+  if (!context->getProperty(property_name, value)) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property missing or invalid");
+  }
+  return value;
+}
+
+std::vector<std::string> listFromCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  std::string property_string;
+  context->getProperty(property_name, property_string);
+  return utils::StringUtils::splitAndTrim(property_string, ",");
+}
+
+std::vector<std::string> listFromRequiredCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  return utils::StringUtils::splitAndTrim(getRequiredPropertyOrThrow(context, property_name), ",");
+}
+
+bool parseBooleanPropertyOrThrow(core::ProcessContext* context, const std::string& property_name) {
+  bool value;
+  std::string value_str = getRequiredPropertyOrThrow(context, property_name);
+  utils::optional<bool> maybe_value = utils::StringUtils::toBool(value_str);
+  if (!maybe_value) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property is invalid: value is " + value_str);
+  }
+  return maybe_value.value();
+}
+
+std::chrono::milliseconds parseTimePropertyMSOrThrow(core::ProcessContext* context, const std::string& property_name) {
+  core::TimeUnit unit;
+  uint64_t time_value_ms;
+  if (!core::Property::StringToTime(getRequiredPropertyOrThrow(context, property_name), time_value_ms, unit) || !core::Property::ConvertTimeUnitToMS(time_value_ms, unit, time_value_ms)) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property missing or invalid");

Review comment:
       Updated the error message.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553321709



##########
File path: extensions/librdkafka/tests/ConsumeKafkaTests.cpp
##########
@@ -0,0 +1,593 @@
+
+/**
+ * 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 "../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(), "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());

Review comment:
       I wonder why I chose `join_pack`. Updated to simple string concatenation.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r540243736



##########
File path: libminifi/include/utils/GeneralUtils.h
##########
@@ -49,6 +49,14 @@ constexpr T intdiv_ceil(T numerator, T denominator) {
       : numerator / denominator + (numerator % denominator != 0));
 }
 
+// from https://stackoverflow.com/questions/15202474
+struct identity {
+    template<typename U>
+    constexpr auto operator()(U&& v) const noexcept -> decltype(std::forward<U>(v)) {
+        return std::forward<U>(v);
+    }
+};
+

Review comment:
       Updated.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553324753



##########
File path: extensions/librdkafka/tests/ConsumeKafkaTests.cpp
##########
@@ -0,0 +1,593 @@
+
+/**
+ * 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 "../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(), "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) {
+    reInitialize();
+
+    // Consumer chain
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+    std::shared_ptr<core::Processor> extract_text  = plan_->addProcessor("ExtractText", "extract_text", {success}, false);
+
+    // Set up connections
+    plan_->addConnection(consume_kafka, success, extract_text);
+    extract_text->setAutoTerminatedRelationships({success});
+
+    const auto bool_to_string = [] (const bool b) -> std::string { return b ? "true" : "false"; };
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::SecurityProtocol.getName(), security_protocol);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), topic_names);
+
+    optional_set_property(consume_kafka, ConsumeKafka::TopicNameFormat.getName(), topic_name_format);
+    optional_set_property(consume_kafka, ConsumeKafka::HonorTransactions.getName(), honor_transactions | utils::map(bool_to_string));
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+    optional_set_property(consume_kafka, ConsumeKafka::OffsetReset.getName(), offset_reset);
+    optional_set_property(consume_kafka, ConsumeKafka::KeyAttributeEncoding.getName(), key_attribute_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageDemarcator.getName(), message_demarcator);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageHeaderEncoding.getName(), message_header_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::HeadersToAddAsAttributes.getName(), headers_to_add_as_attributes);
+    optional_set_property(consume_kafka, ConsumeKafka::DuplicateHeaderHandling.getName(), duplicate_header_handling);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    plan_->setProperty(extract_text, ExtractText::Attribute.getName(), ATTRIBUTE_FOR_CAPTURING_CONTENT);
+
+    if (!expect_config_valid) {
+      const auto& message = messages_on_topic.front();
+      REQUIRE_THROWS(plan_->scheduleProcessor(consume_kafka));
+      return;
+    } else {
+      plan_->scheduleProcessors();
+    }
+
+    std::unique_ptr<rd_kafka_conf_t, utils::rd_kafka_conf_deleter> conf_;
+    std::unique_ptr<rd_kafka_t, utils::rd_kafka_consumer_deleter> consumer_;
+
+    const bool is_transactional = std::count(transaction_events.cbegin(), transaction_events.cend(), KafkaTestProducer::PublishEvent::TRANSACTION_START);
+    const bool transactions_committed = transaction_events.back() == KafkaTestProducer::PublishEvent::TRANSACTION_COMMIT;
+
+    KafkaTestProducer producer(kafka_brokers, PRODUCER_TOPIC, is_transactional);
+    producer.publish_messages_to_topic(messages_on_topic, TEST_MESSAGE_KEY, transaction_events, message_headers, message_header_encoding);
+
+
+    std::vector<std::shared_ptr<core::FlowFile>> flow_files_produced;
+    for (std::size_t num_expected_messages_processed = 0; num_expected_messages_processed < messages_on_topic.size(); num_expected_messages_processed += std::stoi(max_poll_records.value_or("1"))) {
+      plan_->increment_location();
+      if ((honor_transactions && false == honor_transactions.value()) || (is_transactional && !transactions_committed)) {
+        INFO("Non-committed messages received.");
+        REQUIRE(false == plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+        return;
+      }
+      {
+        SCOPED_INFO("ConsumeKafka timed out when waiting to receive the message published to the kafka broker.");
+        REQUIRE(plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+      }
+      std::size_t num_flow_files_produced = plan_->getNumFlowFileProducedByCurrentProcessor();
+      plan_->increment_location();
+      for (std::size_t times_extract_text_run = 0; times_extract_text_run < num_flow_files_produced; ++times_extract_text_run) {
+        plan_->runCurrentProcessor();  // ExtractText
+        std::shared_ptr<core::FlowFile> flow_file = plan_->getFlowFileProducedByCurrentProcessor();
+        for (const auto& exp_header : expect_header_attributes) {
+          SCOPED_INFO("ConsumeKafka did not produce the expected flowfile attribute from message header: " << exp_header.first << ".");
+          const auto header_attr_opt = flow_file->getAttribute(exp_header.first);
+          REQUIRE(header_attr_opt);
+          REQUIRE(exp_header.second == header_attr_opt.value().get());
+        }
+        {
+          SCOPED_INFO("Message key is missing or incorrect (potential encoding mismatch).");
+          REQUIRE(TEST_MESSAGE_KEY == decode_key(flow_file->getAttribute(ConsumeKafka::KAFKA_MESSAGE_KEY_ATTR).value().get(), key_attribute_encoding));
+          REQUIRE("1" == flow_file->getAttribute(ConsumeKafka::KAFKA_COUNT_ATTR).value().get());
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_OFFSET_ATTR));
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_PARTITION_ATTR));
+          REQUIRE(PRODUCER_TOPIC == flow_file->getAttribute(ConsumeKafka::KAFKA_TOPIC_ATTR).value().get());
+        }
+        flow_files_produced.emplace_back(std::move(flow_file));
+      }
+      plan_->reset_location();
+    }
+
+    const auto contentOrderOfFlowFile = [&] (const std::shared_ptr<core::FlowFile>& lhs, const std::shared_ptr<core::FlowFile>& rhs) {
+      return lhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() < rhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get();
+    };
+    {
+      SCOPED_INFO("The flowfiles generated by ConsumeKafka are invalid (probably nullptr).");
+      CHECK_NOTHROW(std::sort(flow_files_produced.begin(), flow_files_produced.end(), contentOrderOfFlowFile));

Review comment:
       I am not sure `REQUIRE_NOTHROW` supports `SCOPED_INFO`. Updated nonetheless, if it fails here, it should be easy to see why.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r534936351



##########
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:
       I think as the error-enum returned is not obvious this is nice to have here. Another point is that one might not immediately think this could be a point of failure. I would have had this code in as debug log, but that would mean that any deleter would have to have an access to a logger.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r577519674



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,578 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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 comma separated list to match against all message headers. Any message header whose name matches an item from the list will be added to the FlowFile "
+      "as an Attribute. If not specified, no Header values will be added as FlowFile attributes. The behaviour on when multiple headers of the same name are present is set using "
+      "the DuplicateHeaderHandling attribute.")
+  ->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 */) {
+  gsl_Expects(context);
+  // 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);
+}
+
+namespace {
+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");
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Failed to create Kafka consumer " + error_msg);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));
+  }
+  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::steady_clock::now();
+  auto elapsed = std::chrono::steady_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());
+    std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>
+      message { rd_kafka_consumer_poll(consumer_.get(), std::chrono::duration_cast<std::chrono::milliseconds>(max_poll_time_milliseconds_ - elapsed).count()), utils::rd_kafka_message_deleter() };
+    if (!message) {
+      break;
+    }
+    if (RD_KAFKA_RESP_ERR_NO_ERROR != message->err) {
+      logger_->log_error("Received message with error %d: %s", message->err, rd_kafka_err2str(message->err));
+      break;
+    }
+    utils::print_kafka_message(message.get(), *logger_);
+    messages.emplace_back(std::move(message));
+    elapsed = std::chrono::steady_clock::now() - start;
+  }
+  return messages;
+}
+
+utils::KafkaEncoding ConsumeKafka::key_attr_encoding_attr_to_enum() const {
+  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() const {
+  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, "\"Message Header Encoding\" property not recognized.");
+}
+
+std::string ConsumeKafka::resolve_duplicate_headers(const std::vector<std::string>& matching_headers) const {
+  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) const {
+  // 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 < 200) {
+      logger_->log_debug("%.*s", static_cast<int>(size), value);
+    } else {
+      logger_->log_debug("%.*s...", 200, 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) const {
+  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_) {
+      ret = stream->write(data_,  gsl::narrow<int>(dataSize_));
+    }
+    return ret;
+  }
+};
+
+void ConsumeKafka::add_kafka_attributes_to_flowfile(std::shared_ptr<FlowFileRecord>& flow_file, const rd_kafka_message_t* message) const {
+  // 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, 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));
+}
+
+utils::optional<std::vector<std::shared_ptr<FlowFileRecord>>> ConsumeKafka::transform_pending_messages_into_flowfiles(core::ProcessSession* session) const {

Review comment:
       I thought so, but it improves readability and therefore safety when you can verify that locally.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] fgerlits commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
fgerlits commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r540258016



##########
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) {
+  std::string value;
+  if (!context->getProperty(property_name, value)) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property missing or invalid");
+  }
+  return value;
+}
+
+std::vector<std::string> listFromCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  std::string property_string;
+  context->getProperty(property_name, property_string);
+  return utils::StringUtils::splitAndTrim(property_string, ",");
+}
+
+std::vector<std::string> listFromRequiredCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  return utils::StringUtils::splitAndTrim(getRequiredPropertyOrThrow(context, property_name), ",");
+}
+
+bool parseBooleanPropertyOrThrow(core::ProcessContext* context, const std::string& property_name) {
+  bool value;
+  std::string value_str = getRequiredPropertyOrThrow(context, property_name);
+  return utils::StringUtils::StringToBool(value_str, value);

Review comment:
       This introduces a new case where we use the return value of `StringToBool` as the conversion result.  It might be better/clearer to use the new `StringUtils::toBool()` function, and return an `optional<bool>`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] fgerlits commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
fgerlits commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r576623105



##########
File path: libminifi/test/TestBase.cpp
##########
@@ -62,78 +63,51 @@ TestPlan::~TestPlan() {
   for (auto& processor : configured_processors_) {
     processor->setScheduledState(core::ScheduledState::STOPPED);
   }
+  for (auto& connection : relationships_) {
+    // This is a patch solving circular references between processors and connections
+    connection->setSource(nullptr);
+    connection->setDestination(nullptr);
+  }
   controller_services_provider_->clearControllerServices();
 }
 
 std::shared_ptr<core::Processor> TestPlan::addProcessor(const std::shared_ptr<core::Processor> &processor, const std::string &name, const std::initializer_list<core::Relationship>& relationships,
-                                                        bool linkToPrevious) {
+    bool linkToPrevious) {
   if (finalized) {
     return nullptr;
   }
   std::lock_guard<std::recursive_mutex> guard(mutex);
-
   utils::Identifier uuid = utils::IdGenerator::getIdGenerator()->generate();
-
   processor->setStreamFactory(stream_factory);
   // initialize the processor
   processor->initialize();
   processor->setFlowIdentifier(flow_version_->getFlowIdentifier());
-
   processor_mapping_[processor->getUUID()] = processor;
-
   if (!linkToPrevious) {
     termination_ = *(relationships.begin());
   } else {
     std::shared_ptr<core::Processor> last = processor_queue_.back();
-
     if (last == nullptr) {
       last = processor;
       termination_ = *(relationships.begin());
     }
-
-    std::stringstream connection_name;
-    connection_name << last->getUUIDStr() << "-to-" << processor->getUUIDStr();
-    logger_->log_info("Creating %s connection for proc %d", connection_name.str(), processor_queue_.size() + 1);
-    std::shared_ptr<minifi::Connection> connection = std::make_shared<minifi::Connection>(flow_repo_, content_repo_, connection_name.str());
-
     for (const auto& relationship : relationships) {
-      connection->addRelationship(relationship);
-    }
-
-    // link the connections so that we can test results at the end for this
-    connection->setSource(last);
-    connection->setDestination(processor);
-
-    connection->setSourceUUID(last->getUUID());
-    connection->setDestinationUUID(processor->getUUID());
-    last->addConnection(connection);
-    if (last != processor) {
-      processor->addConnection(connection);
+      addConnection(last, relationship, processor);

Review comment:
       This change was reverted in https://github.com/apache/nifi-minifi-cpp/pull/940/commits/33e84f4f31b803a0cf0f212287c701d922295059.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553202896



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  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() const {
+  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() const {
+  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) const {
+  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) const {
+  // 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) const {
+  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_);

Review comment:
       Yep, during the implementation of ConsumeKafka, this stream writer suddenly changed and I thought just matching the types is good enough. It really makes no sense to check the sign of an `uint64_t`. Updated to use `gsl::narrow`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] fgerlits commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
fgerlits commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r552515195



##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);
+  }
+}
+
+void print_topics_list(std::shared_ptr<logging::Logger> logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger->log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld]",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, const std::shared_ptr<logging::Logger>& logger) {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    const std::string error_msg = "ConsumeKafka: received error message from broker. Librdkafka error msg: " + std::string(rd_kafka_err2str(rkmessage->err));
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, error_msg);
+  }
+  std::string topicName = rd_kafka_topic_name(rkmessage->rkt);
+  std::string message(reinterpret_cast<char*>(rkmessage->payload), rkmessage->len);
+  const char* key = reinterpret_cast<const char*>(rkmessage->key);
+  const std::size_t key_len = rkmessage->key_len;
+  rd_kafka_timestamp_type_t tstype;
+  int64_t timestamp;
+  timestamp = rd_kafka_message_timestamp(rkmessage, &tstype);
+  const char *tsname = "?";
+  if (tstype != RD_KAFKA_TIMESTAMP_NOT_AVAILABLE) {
+    if (tstype == RD_KAFKA_TIMESTAMP_CREATE_TIME) {
+      tsname = "create time";
+    } else if (tstype == RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME) {
+      tsname = "log append time";
+    }
+  }
+  const int64_t seconds_since_timestamp = timestamp ? static_cast<int64_t>(time(NULL)) - static_cast<int64_t>(timestamp / 1000) : 0;
+
+  std::string headers_as_string;
+  rd_kafka_headers_t* hdrs;
+  const rd_kafka_resp_err_t get_header_response = rd_kafka_message_headers(rkmessage, &hdrs);
+  if (RD_KAFKA_RESP_ERR_NO_ERROR == get_header_response) {
+    std::vector<std::string> header_list;
+    kafka_headers_for_each(hdrs, [&] (const std::string& key, const std::string& val) { header_list.emplace_back(key + ": " + val); });
+    headers_as_string = StringUtils::join(", ", header_list);
+  } else if (RD_KAFKA_RESP_ERR__NOENT != 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::string message_as_string;
+  message_as_string += "[Topic](" + topicName + "), ";
+  message_as_string += "[Key](" + (key != nullptr ? std::string(key, key_len) : std::string("[None]")) + "), ";
+  message_as_string += "[Offset](" +  std::to_string(rkmessage->offset) + "), ";
+  message_as_string += "[Message Length](" + std::to_string(rkmessage->len) + "), ";
+  message_as_string += "[Timestamp](" + std::string(tsname) + " " + std::to_string(timestamp) + " (" + std::to_string(seconds_since_timestamp) + " s ago)), ";

Review comment:
       if we could not get a timestamp, it may be better to not print it instead of printing "[Timestamp](? -1 (0 s ago))"

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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);

Review comment:
       Since `print_topics_list` logs the topics on Debug level, it could be wrapped in an `if (logger->should_log(...))`.

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);
+  }
+}
+
+void print_topics_list(std::shared_ptr<logging::Logger> logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger->log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld]",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, const std::shared_ptr<logging::Logger>& logger) {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    const std::string error_msg = "ConsumeKafka: received error message from broker. Librdkafka error msg: " + std::string(rd_kafka_err2str(rkmessage->err));
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, error_msg);
+  }
+  std::string topicName = rd_kafka_topic_name(rkmessage->rkt);
+  std::string message(reinterpret_cast<char*>(rkmessage->payload), rkmessage->len);
+  const char* key = reinterpret_cast<const char*>(rkmessage->key);
+  const std::size_t key_len = rkmessage->key_len;
+  rd_kafka_timestamp_type_t tstype;
+  int64_t timestamp;
+  timestamp = rd_kafka_message_timestamp(rkmessage, &tstype);
+  const char *tsname = "?";
+  if (tstype != RD_KAFKA_TIMESTAMP_NOT_AVAILABLE) {
+    if (tstype == RD_KAFKA_TIMESTAMP_CREATE_TIME) {
+      tsname = "create time";
+    } else if (tstype == RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME) {
+      tsname = "log append time";
+    }
+  }
+  const int64_t seconds_since_timestamp = timestamp ? static_cast<int64_t>(time(NULL)) - static_cast<int64_t>(timestamp / 1000) : 0;

Review comment:
       The documentation of `rd_kafka_message_timestamp()` says it returns `-1` on errors, so this should be `... = (timestamp != -1) ? ...`

##########
File path: extensions/librdkafka/docker_tests/CMakeLists.txt
##########
@@ -0,0 +1,36 @@
+#

Review comment:
       this file would have a better place in the later PR which will contain the docker tests

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);
+  }
+}
+
+void print_topics_list(std::shared_ptr<logging::Logger> logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger->log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld]",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, const std::shared_ptr<logging::Logger>& logger) {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    const std::string error_msg = "ConsumeKafka: received error message from broker. Librdkafka error msg: " + std::string(rd_kafka_err2str(rkmessage->err));
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, error_msg);
+  }
+  std::string topicName = rd_kafka_topic_name(rkmessage->rkt);
+  std::string message(reinterpret_cast<char*>(rkmessage->payload), rkmessage->len);
+  const char* key = reinterpret_cast<const char*>(rkmessage->key);
+  const std::size_t key_len = rkmessage->key_len;
+  rd_kafka_timestamp_type_t tstype;
+  int64_t timestamp;
+  timestamp = rd_kafka_message_timestamp(rkmessage, &tstype);
+  const char *tsname = "?";
+  if (tstype != RD_KAFKA_TIMESTAMP_NOT_AVAILABLE) {
+    if (tstype == RD_KAFKA_TIMESTAMP_CREATE_TIME) {
+      tsname = "create time";
+    } else if (tstype == RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME) {
+      tsname = "log append time";
+    }
+  }
+  const int64_t seconds_since_timestamp = timestamp ? static_cast<int64_t>(time(NULL)) - static_cast<int64_t>(timestamp / 1000) : 0;
+
+  std::string headers_as_string;
+  rd_kafka_headers_t* hdrs;
+  const rd_kafka_resp_err_t get_header_response = rd_kafka_message_headers(rkmessage, &hdrs);
+  if (RD_KAFKA_RESP_ERR_NO_ERROR == get_header_response) {
+    std::vector<std::string> header_list;
+    kafka_headers_for_each(hdrs, [&] (const std::string& key, const std::string& val) { header_list.emplace_back(key + ": " + val); });
+    headers_as_string = StringUtils::join(", ", header_list);
+  } else if (RD_KAFKA_RESP_ERR__NOENT != 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::string message_as_string;
+  message_as_string += "[Topic](" + topicName + "), ";
+  message_as_string += "[Key](" + (key != nullptr ? std::string(key, key_len) : std::string("[None]")) + "), ";
+  message_as_string += "[Offset](" +  std::to_string(rkmessage->offset) + "), ";
+  message_as_string += "[Message Length](" + std::to_string(rkmessage->len) + "), ";
+  message_as_string += "[Timestamp](" + std::string(tsname) + " " + std::to_string(timestamp) + " (" + std::to_string(seconds_since_timestamp) + " s ago)), ";
+  message_as_string += "[Headers](";
+  message_as_string += headers_as_string + "\n";

Review comment:
       Should this `"\n"` be `"), "`?  https://xkcd.com/859/

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };

Review comment:
       Same problem as at ConsumeKafka.cpp:333:  this will create a string of length 512 with embedded nulls.  If we are sure Kafka returns a null-terminated error message, we can do `const std::string error_msg{errstr.data()};`




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r578246029



##########
File path: extensions/librdkafka/rdkafka_utils.h
##########
@@ -0,0 +1,105 @@
+/**
+ * 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 "utils/gsl.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;
+    // }
+    rd_kafka_destroy(ptr);
+  }
+};
+
+struct rd_kafka_consumer_deleter {
+  void operator()(rd_kafka_t* ptr) const noexcept {
+    rd_kafka_consumer_close(ptr);
+    rd_kafka_destroy(ptr);
+  }
+};
+
+struct rd_kafka_topic_partition_list_deleter {
+  void operator()(rd_kafka_topic_partition_list_t* ptr) const noexcept { rd_kafka_topic_partition_list_destroy(ptr); }
+};
+
+struct rd_kafka_topic_conf_deleter {
+  void operator()(rd_kafka_topic_conf_t* ptr) const noexcept { rd_kafka_topic_conf_destroy(ptr); }
+};
+struct rd_kafka_topic_deleter {
+  void operator()(rd_kafka_topic_t* ptr) const noexcept { rd_kafka_topic_destroy(ptr); }
+};
+
+struct rd_kafka_message_deleter {
+  void operator()(rd_kafka_message_t* ptr) const noexcept { rd_kafka_message_destroy(ptr); }
+};
+
+struct rd_kafka_headers_deleter {
+  void operator()(rd_kafka_headers_t* ptr) const noexcept { rd_kafka_headers_destroy(ptr); }
+};
+
+template <typename T>
+void kafka_headers_for_each(const rd_kafka_headers_t* headers, T key_value_handle) {

Review comment:
       Updated.

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,121 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error: " + error_msg);
+  }
+}
+
+void print_topics_list(logging::Logger& logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger.log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset: %" PRId64 ".",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+std::string get_human_readable_kafka_message_timestamp(const rd_kafka_message_t* rkmessage) {
+  rd_kafka_timestamp_type_t tstype;
+  int64_t timestamp;
+  timestamp = rd_kafka_message_timestamp(rkmessage, &tstype);
+  const char *tsname = "?";
+  if (tstype == RD_KAFKA_TIMESTAMP_CREATE_TIME) {
+    tsname = "create time";
+  } else if (tstype == RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME) {
+    tsname = "log append time";
+  }
+  const int64_t seconds_since_timestamp = timestamp == -1 ? 0 : static_cast<int64_t>(time(NULL)) - static_cast<int64_t>(timestamp / 1000);
+  return {"[Timestamp](" + std::string(tsname) + " " + std::to_string(timestamp) + " (" + std::to_string(seconds_since_timestamp) + " s ago)"};
+}
+
+std::string get_human_readable_kafka_message_headers(const rd_kafka_message_t* rkmessage, logging::Logger& logger) {
+  rd_kafka_headers_t* hdrs;
+  const rd_kafka_resp_err_t get_header_response = rd_kafka_message_headers(rkmessage, &hdrs);
+  if (RD_KAFKA_RESP_ERR_NO_ERROR == get_header_response) {
+    std::vector<std::string> header_list;
+    kafka_headers_for_each(hdrs, [&] (const std::string& key, gsl::span<const char> val) { header_list.emplace_back(key + ": " + std::string{ val.data(), val.size() }); });
+    return StringUtils::join(", ", header_list);
+  }
+  if (RD_KAFKA_RESP_ERR__NOENT == get_header_response) {
+    return "[None]";
+  }
+  logger.log_error("Failed to fetch message headers: %d: %s", rd_kafka_last_error(), rd_kafka_err2str(rd_kafka_last_error()));
+  return "[Error]";
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, logging::Logger& logger) {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    const std::string error_msg = "ConsumeKafka: received error message from broker. Librdkafka error msg: " + std::string(rd_kafka_err2str(rkmessage->err));
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, error_msg);
+  }
+  std::string topicName = rd_kafka_topic_name(rkmessage->rkt);
+  std::string message(reinterpret_cast<char*>(rkmessage->payload), rkmessage->len);
+  const char* key = reinterpret_cast<const char*>(rkmessage->key);
+  const std::size_t key_len = rkmessage->key_len;
+
+  std::string message_as_string;
+  message_as_string += "[Topic](" + topicName + "), ";
+  message_as_string += "[Key](" + (key != nullptr ? std::string(key, key_len) : std::string("[None]")) + "), ";
+  message_as_string += "[Offset](" +  std::to_string(rkmessage->offset) + "), ";
+  message_as_string += "[Message Length](" + std::to_string(rkmessage->len) + "), ";
+  message_as_string += get_human_readable_kafka_message_timestamp(rkmessage) + "), ";
+  message_as_string += "[Headers](";
+  message_as_string += get_human_readable_kafka_message_headers(rkmessage, logger) + ")";
+  message_as_string += "[Payload](" + message + ")";
+
+  logger.log_debug("Message: %s", message_as_string.c_str());
+}
+
+std::string get_encoded_string(const std::string& input, KafkaEncoding encoding) {
+  switch (encoding) {
+    case KafkaEncoding::UTF8:
+      return input;
+    case KafkaEncoding::HEX:
+      return StringUtils::to_hex(input, /* uppercase = */ true);
+  }
+  throw std::runtime_error("Invalid encoding selected: " + input);
+}
+
+optional<std::string> get_encoded_message_key(const rd_kafka_message_t* message, KafkaEncoding encoding) {

Review comment:
       Updated.

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,121 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error: " + error_msg);
+  }
+}
+
+void print_topics_list(logging::Logger& logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger.log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset: %" PRId64 ".",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+std::string get_human_readable_kafka_message_timestamp(const rd_kafka_message_t* rkmessage) {
+  rd_kafka_timestamp_type_t tstype;
+  int64_t timestamp;
+  timestamp = rd_kafka_message_timestamp(rkmessage, &tstype);
+  const char *tsname = "?";
+  if (tstype == RD_KAFKA_TIMESTAMP_CREATE_TIME) {
+    tsname = "create time";
+  } else if (tstype == RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME) {
+    tsname = "log append time";
+  }
+  const int64_t seconds_since_timestamp = timestamp == -1 ? 0 : static_cast<int64_t>(time(NULL)) - static_cast<int64_t>(timestamp / 1000);
+  return {"[Timestamp](" + std::string(tsname) + " " + std::to_string(timestamp) + " (" + std::to_string(seconds_since_timestamp) + " s ago)"};
+}
+
+std::string get_human_readable_kafka_message_headers(const rd_kafka_message_t* rkmessage, logging::Logger& logger) {
+  rd_kafka_headers_t* hdrs;
+  const rd_kafka_resp_err_t get_header_response = rd_kafka_message_headers(rkmessage, &hdrs);
+  if (RD_KAFKA_RESP_ERR_NO_ERROR == get_header_response) {
+    std::vector<std::string> header_list;
+    kafka_headers_for_each(hdrs, [&] (const std::string& key, gsl::span<const char> val) { header_list.emplace_back(key + ": " + std::string{ val.data(), val.size() }); });
+    return StringUtils::join(", ", header_list);
+  }
+  if (RD_KAFKA_RESP_ERR__NOENT == get_header_response) {
+    return "[None]";
+  }
+  logger.log_error("Failed to fetch message headers: %d: %s", rd_kafka_last_error(), rd_kafka_err2str(rd_kafka_last_error()));
+  return "[Error]";
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, logging::Logger& logger) {

Review comment:
       Updated.

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,121 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error: " + error_msg);
+  }
+}
+
+void print_topics_list(logging::Logger& logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger.log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset: %" PRId64 ".",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+std::string get_human_readable_kafka_message_timestamp(const rd_kafka_message_t* rkmessage) {
+  rd_kafka_timestamp_type_t tstype;
+  int64_t timestamp;
+  timestamp = rd_kafka_message_timestamp(rkmessage, &tstype);
+  const char *tsname = "?";
+  if (tstype == RD_KAFKA_TIMESTAMP_CREATE_TIME) {
+    tsname = "create time";
+  } else if (tstype == RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME) {
+    tsname = "log append time";
+  }
+  const int64_t seconds_since_timestamp = timestamp == -1 ? 0 : static_cast<int64_t>(time(NULL)) - static_cast<int64_t>(timestamp / 1000);
+  return {"[Timestamp](" + std::string(tsname) + " " + std::to_string(timestamp) + " (" + std::to_string(seconds_since_timestamp) + " s ago)"};
+}
+
+std::string get_human_readable_kafka_message_headers(const rd_kafka_message_t* rkmessage, logging::Logger& logger) {

Review comment:
       Updated.

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,121 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error: " + error_msg);
+  }
+}
+
+void print_topics_list(logging::Logger& logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger.log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset: %" PRId64 ".",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+std::string get_human_readable_kafka_message_timestamp(const rd_kafka_message_t* rkmessage) {

Review comment:
       Updated.

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,121 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error: " + error_msg);
+  }
+}
+
+void print_topics_list(logging::Logger& logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {

Review comment:
       Updated.

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,121 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {

Review comment:
       Updated.

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,570 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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 comma separated list to match against all message headers. Any message header whose name matches an item from the list will be added to the FlowFile "
+      "as an Attribute. If not specified, no Header values will be added as FlowFile attributes. The behaviour on when multiple headers of the same name are present is set using "
+      "the DuplicateHeaderHandling attribute.")
+  ->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 */) {
+  gsl_Expects(context);
+  // 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);
+}
+
+namespace {
+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:");
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Failed to create Kafka consumer " + error_msg);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));
+  }
+  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::steady_clock::now();
+  auto elapsed = std::chrono::steady_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());
+    std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>
+      message { rd_kafka_consumer_poll(consumer_.get(), std::chrono::duration_cast<std::chrono::milliseconds>(max_poll_time_milliseconds_ - elapsed).count()), utils::rd_kafka_message_deleter() };
+    if (!message) {
+      break;
+    }
+    if (RD_KAFKA_RESP_ERR_NO_ERROR != message->err) {
+      logger_->log_error("Received message with error %d: %s", message->err, rd_kafka_err2str(message->err));
+      break;
+    }
+    utils::print_kafka_message(message.get(), *logger_);
+    messages.emplace_back(std::move(message));
+    elapsed = std::chrono::steady_clock::now() - start;
+  }
+  return messages;
+}
+
+utils::KafkaEncoding ConsumeKafka::key_attr_encoding_attr_to_enum() const {
+  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() const {
+  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, "\"Message Header Encoding\" property not recognized.");
+}
+
+std::string ConsumeKafka::resolve_duplicate_headers(const std::vector<std::string>& matching_headers) const {
+  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) const {
+  // 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 < 200) {
+      logger_->log_debug("%.*s", static_cast<int>(size), value);
+    } else {
+      logger_->log_debug("%.*s...", 200, 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) const {
+  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;
+}
+
+void ConsumeKafka::add_kafka_attributes_to_flowfile(std::shared_ptr<FlowFileRecord>& flow_file, const rd_kafka_message_t* message) const {

Review comment:
       Updated.

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,570 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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 comma separated list to match against all message headers. Any message header whose name matches an item from the list will be added to the FlowFile "
+      "as an Attribute. If not specified, no Header values will be added as FlowFile attributes. The behaviour on when multiple headers of the same name are present is set using "
+      "the DuplicateHeaderHandling attribute.")
+  ->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 */) {
+  gsl_Expects(context);
+  // 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);
+}
+
+namespace {
+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:");
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Failed to create Kafka consumer " + error_msg);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));
+  }
+  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::steady_clock::now();
+  auto elapsed = std::chrono::steady_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());
+    std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>
+      message { rd_kafka_consumer_poll(consumer_.get(), std::chrono::duration_cast<std::chrono::milliseconds>(max_poll_time_milliseconds_ - elapsed).count()), utils::rd_kafka_message_deleter() };
+    if (!message) {
+      break;
+    }
+    if (RD_KAFKA_RESP_ERR_NO_ERROR != message->err) {
+      logger_->log_error("Received message with error %d: %s", message->err, rd_kafka_err2str(message->err));
+      break;
+    }
+    utils::print_kafka_message(message.get(), *logger_);
+    messages.emplace_back(std::move(message));
+    elapsed = std::chrono::steady_clock::now() - start;
+  }
+  return messages;
+}
+
+utils::KafkaEncoding ConsumeKafka::key_attr_encoding_attr_to_enum() const {
+  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() const {
+  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, "\"Message Header Encoding\" property not recognized.");
+}
+
+std::string ConsumeKafka::resolve_duplicate_headers(const std::vector<std::string>& matching_headers) const {
+  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) const {
+  // 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 < 200) {
+      logger_->log_debug("%.*s", static_cast<int>(size), value);
+    } else {
+      logger_->log_debug("%.*s...", 200, 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) const {

Review comment:
       Updated.

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,570 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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 comma separated list to match against all message headers. Any message header whose name matches an item from the list will be added to the FlowFile "
+      "as an Attribute. If not specified, no Header values will be added as FlowFile attributes. The behaviour on when multiple headers of the same name are present is set using "
+      "the DuplicateHeaderHandling attribute.")
+  ->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 */) {
+  gsl_Expects(context);
+  // 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);
+}
+
+namespace {
+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:");
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Failed to create Kafka consumer " + error_msg);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));
+  }
+  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::steady_clock::now();
+  auto elapsed = std::chrono::steady_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());
+    std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>
+      message { rd_kafka_consumer_poll(consumer_.get(), std::chrono::duration_cast<std::chrono::milliseconds>(max_poll_time_milliseconds_ - elapsed).count()), utils::rd_kafka_message_deleter() };
+    if (!message) {
+      break;
+    }
+    if (RD_KAFKA_RESP_ERR_NO_ERROR != message->err) {
+      logger_->log_error("Received message with error %d: %s", message->err, rd_kafka_err2str(message->err));
+      break;
+    }
+    utils::print_kafka_message(message.get(), *logger_);
+    messages.emplace_back(std::move(message));
+    elapsed = std::chrono::steady_clock::now() - start;
+  }
+  return messages;
+}
+
+utils::KafkaEncoding ConsumeKafka::key_attr_encoding_attr_to_enum() const {
+  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() const {
+  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, "\"Message Header Encoding\" property not recognized.");
+}
+
+std::string ConsumeKafka::resolve_duplicate_headers(const std::vector<std::string>& matching_headers) const {
+  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) const {

Review comment:
       Updated.

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,570 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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 comma separated list to match against all message headers. Any message header whose name matches an item from the list will be added to the FlowFile "
+      "as an Attribute. If not specified, no Header values will be added as FlowFile attributes. The behaviour on when multiple headers of the same name are present is set using "
+      "the DuplicateHeaderHandling attribute.")
+  ->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 */) {
+  gsl_Expects(context);
+  // 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);
+}
+
+namespace {
+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:");
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Failed to create Kafka consumer " + error_msg);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {

Review comment:
       Updated




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553168700



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);

Review comment:
       [[DoxyGen]](https://docs.confluent.io/2.0.0/clients/librdkafka/rdkafka_8h.html#a0ebe15e9d0f39ccc84e9686f0fcf46f1), [[Source]](https://github.com/edenhill/librdkafka/blob/v1.5.2/src/rdkafka.h#L3565-L3568). There is also a test case in the `[Topic]` tag that checks for matching against regex topics.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553317390



##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);
+  }
+}
+
+void print_topics_list(std::shared_ptr<logging::Logger> logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger->log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld]",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, const std::shared_ptr<logging::Logger>& logger) {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    const std::string error_msg = "ConsumeKafka: received error message from broker. Librdkafka error msg: " + std::string(rd_kafka_err2str(rkmessage->err));
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, error_msg);
+  }
+  std::string topicName = rd_kafka_topic_name(rkmessage->rkt);
+  std::string message(reinterpret_cast<char*>(rkmessage->payload), rkmessage->len);
+  const char* key = reinterpret_cast<const char*>(rkmessage->key);
+  const std::size_t key_len = rkmessage->key_len;
+  rd_kafka_timestamp_type_t tstype;
+  int64_t timestamp;
+  timestamp = rd_kafka_message_timestamp(rkmessage, &tstype);
+  const char *tsname = "?";
+  if (tstype != RD_KAFKA_TIMESTAMP_NOT_AVAILABLE) {
+    if (tstype == RD_KAFKA_TIMESTAMP_CREATE_TIME) {
+      tsname = "create time";
+    } else if (tstype == RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME) {
+      tsname = "log append time";
+    }
+  }
+  const int64_t seconds_since_timestamp = timestamp ? static_cast<int64_t>(time(NULL)) - static_cast<int64_t>(timestamp / 1000) : 0;
+
+  std::string headers_as_string;
+  rd_kafka_headers_t* hdrs;
+  const rd_kafka_resp_err_t get_header_response = rd_kafka_message_headers(rkmessage, &hdrs);
+  if (RD_KAFKA_RESP_ERR_NO_ERROR == get_header_response) {
+    std::vector<std::string> header_list;
+    kafka_headers_for_each(hdrs, [&] (const std::string& key, const std::string& val) { header_list.emplace_back(key + ": " + val); });
+    headers_as_string = StringUtils::join(", ", header_list);
+  } else if (RD_KAFKA_RESP_ERR__NOENT != 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::string message_as_string;
+  message_as_string += "[Topic](" + topicName + "), ";
+  message_as_string += "[Key](" + (key != nullptr ? std::string(key, key_len) : std::string("[None]")) + "), ";
+  message_as_string += "[Offset](" +  std::to_string(rkmessage->offset) + "), ";
+  message_as_string += "[Message Length](" + std::to_string(rkmessage->len) + "), ";
+  message_as_string += "[Timestamp](" + std::string(tsname) + " " + std::to_string(timestamp) + " (" + std::to_string(seconds_since_timestamp) + " s ago)), ";
+  message_as_string += "[Headers](";
+  message_as_string += headers_as_string + "\n";

Review comment:
       Oh no. What have I done.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553194936



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  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());

Review comment:
       Good catch. Indeed, we don't delete this in case we received a message but it is erroneous. Updated.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r567680555



##########
File path: libminifi/test/TestBase.cpp
##########
@@ -62,78 +63,51 @@ TestPlan::~TestPlan() {
   for (auto& processor : configured_processors_) {
     processor->setScheduledState(core::ScheduledState::STOPPED);
   }
+  for (auto& connection : relationships_) {
+    // This is a patch solving circular references between processors and connections
+    connection->setSource(nullptr);
+    connection->setDestination(nullptr);
+  }
   controller_services_provider_->clearControllerServices();
 }
 
 std::shared_ptr<core::Processor> TestPlan::addProcessor(const std::shared_ptr<core::Processor> &processor, const std::string &name, const std::initializer_list<core::Relationship>& relationships,
-                                                        bool linkToPrevious) {
+    bool linkToPrevious) {
   if (finalized) {
     return nullptr;
   }
   std::lock_guard<std::recursive_mutex> guard(mutex);
-
   utils::Identifier uuid = utils::IdGenerator::getIdGenerator()->generate();
-
   processor->setStreamFactory(stream_factory);
   // initialize the processor
   processor->initialize();
   processor->setFlowIdentifier(flow_version_->getFlowIdentifier());
-
   processor_mapping_[processor->getUUID()] = processor;
-
   if (!linkToPrevious) {
     termination_ = *(relationships.begin());
   } else {
     std::shared_ptr<core::Processor> last = processor_queue_.back();
-
     if (last == nullptr) {
       last = processor;
       termination_ = *(relationships.begin());
     }
-
-    std::stringstream connection_name;
-    connection_name << last->getUUIDStr() << "-to-" << processor->getUUIDStr();
-    logger_->log_info("Creating %s connection for proc %d", connection_name.str(), processor_queue_.size() + 1);
-    std::shared_ptr<minifi::Connection> connection = std::make_shared<minifi::Connection>(flow_repo_, content_repo_, connection_name.str());
-
     for (const auto& relationship : relationships) {
-      connection->addRelationship(relationship);
-    }
-
-    // link the connections so that we can test results at the end for this
-    connection->setSource(last);
-    connection->setDestination(processor);
-
-    connection->setSourceUUID(last->getUUID());
-    connection->setDestinationUUID(processor->getUUID());
-    last->addConnection(connection);
-    if (last != processor) {
-      processor->addConnection(connection);
+      addConnection(last, relationship, processor);

Review comment:
       There is a difference between 1 connection for 3 relationships and 3 connections for 3 relationships. The latter may or may not be fine, so it's better not to change the behavior without a good reason.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r534927466



##########
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:
       Agreed, consider it done :)




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553911182



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,582 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger->should_log(core::logging::LOG_LEVEL::info)) {
+        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
+      if (logger->should_log(core::logging::LOG_LEVEL::info)) {

Review comment:
       No reason, I made a mistake. Corrected now.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553186818



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+      throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker.");

Review comment:
       Added error/loggging to both places.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553316698



##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);
+  }
+}
+
+void print_topics_list(std::shared_ptr<logging::Logger> logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger->log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld]",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, const std::shared_ptr<logging::Logger>& logger) {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    const std::string error_msg = "ConsumeKafka: received error message from broker. Librdkafka error msg: " + std::string(rd_kafka_err2str(rkmessage->err));
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, error_msg);
+  }
+  std::string topicName = rd_kafka_topic_name(rkmessage->rkt);
+  std::string message(reinterpret_cast<char*>(rkmessage->payload), rkmessage->len);
+  const char* key = reinterpret_cast<const char*>(rkmessage->key);
+  const std::size_t key_len = rkmessage->key_len;
+  rd_kafka_timestamp_type_t tstype;
+  int64_t timestamp;
+  timestamp = rd_kafka_message_timestamp(rkmessage, &tstype);
+  const char *tsname = "?";
+  if (tstype != RD_KAFKA_TIMESTAMP_NOT_AVAILABLE) {
+    if (tstype == RD_KAFKA_TIMESTAMP_CREATE_TIME) {
+      tsname = "create time";
+    } else if (tstype == RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME) {
+      tsname = "log append time";
+    }
+  }
+  const int64_t seconds_since_timestamp = timestamp ? static_cast<int64_t>(time(NULL)) - static_cast<int64_t>(timestamp / 1000) : 0;

Review comment:
       Good catch, updated.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553312121



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  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() const {
+  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() const {
+  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) const {
+  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) const {
+  // 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) const {
+  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;
+  }
+};
+
+void ConsumeKafka::add_kafka_attributes_to_flowfile(std::shared_ptr<FlowFileRecord>& flow_file, const rd_kafka_message_t* message) const {
+  // 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, 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));
+}
+
+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) const {
+  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 {};

Review comment:
       Update:
   Reworked this part. Now I/O failures are handled by storing non-written messages and attempting to write them on the start of the next onTrigger.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r577719214



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,578 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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 comma separated list to match against all message headers. Any message header whose name matches an item from the list will be added to the FlowFile "
+      "as an Attribute. If not specified, no Header values will be added as FlowFile attributes. The behaviour on when multiple headers of the same name are present is set using "
+      "the DuplicateHeaderHandling attribute.")
+  ->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 */) {
+  gsl_Expects(context);
+  // 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);
+}
+
+namespace {
+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");
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Failed to create Kafka consumer " + error_msg);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));
+  }
+  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::steady_clock::now();
+  auto elapsed = std::chrono::steady_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());
+    std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>
+      message { rd_kafka_consumer_poll(consumer_.get(), std::chrono::duration_cast<std::chrono::milliseconds>(max_poll_time_milliseconds_ - elapsed).count()), utils::rd_kafka_message_deleter() };
+    if (!message) {
+      break;
+    }
+    if (RD_KAFKA_RESP_ERR_NO_ERROR != message->err) {
+      logger_->log_error("Received message with error %d: %s", message->err, rd_kafka_err2str(message->err));
+      break;
+    }
+    utils::print_kafka_message(message.get(), *logger_);
+    messages.emplace_back(std::move(message));
+    elapsed = std::chrono::steady_clock::now() - start;
+  }
+  return messages;
+}
+
+utils::KafkaEncoding ConsumeKafka::key_attr_encoding_attr_to_enum() const {
+  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() const {
+  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, "\"Message Header Encoding\" property not recognized.");
+}
+
+std::string ConsumeKafka::resolve_duplicate_headers(const std::vector<std::string>& matching_headers) const {
+  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) const {
+  // 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 < 200) {
+      logger_->log_debug("%.*s", static_cast<int>(size), value);
+    } else {
+      logger_->log_debug("%.*s...", 200, 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) const {
+  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_) {
+      ret = stream->write(data_,  gsl::narrow<int>(dataSize_));
+    }
+    return ret;
+  }
+};
+
+void ConsumeKafka::add_kafka_attributes_to_flowfile(std::shared_ptr<FlowFileRecord>& flow_file, const rd_kafka_message_t* message) const {
+  // 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, 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));
+}
+
+utils::optional<std::vector<std::shared_ptr<FlowFileRecord>>> ConsumeKafka::transform_pending_messages_into_flowfiles(core::ProcessSession* session) const {

Review comment:
       Updated the helper functions to take session by reference.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553313955



##########
File path: extensions/librdkafka/docker_tests/CMakeLists.txt
##########
@@ -0,0 +1,36 @@
+#

Review comment:
       These are manual tests that are ran for the processor. I would not want a PR with a processor that does not involve the testing done for it.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r567646819



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,579 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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 */) {

Review comment:
       Is it the responsibility of the processor to ensure it is schedu in a valid context?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553906593



##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);
+  }
+}
+
+void print_topics_list(std::shared_ptr<logging::Logger> logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger->log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld]",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, const std::shared_ptr<logging::Logger>& logger) {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    const std::string error_msg = "ConsumeKafka: received error message from broker. Librdkafka error msg: " + std::string(rd_kafka_err2str(rkmessage->err));
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, error_msg);
+  }
+  std::string topicName = rd_kafka_topic_name(rkmessage->rkt);
+  std::string message(reinterpret_cast<char*>(rkmessage->payload), rkmessage->len);
+  const char* key = reinterpret_cast<const char*>(rkmessage->key);
+  const std::size_t key_len = rkmessage->key_len;
+  rd_kafka_timestamp_type_t tstype;
+  int64_t timestamp;
+  timestamp = rd_kafka_message_timestamp(rkmessage, &tstype);
+  const char *tsname = "?";
+  if (tstype != RD_KAFKA_TIMESTAMP_NOT_AVAILABLE) {
+    if (tstype == RD_KAFKA_TIMESTAMP_CREATE_TIME) {
+      tsname = "create time";
+    } else if (tstype == RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME) {
+      tsname = "log append time";
+    }
+  }
+  const int64_t seconds_since_timestamp = timestamp ? static_cast<int64_t>(time(NULL)) - static_cast<int64_t>(timestamp / 1000) : 0;

Review comment:
       It does print the absolute one as well.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553882120



##########
File path: libminifi/test/TestBase.cpp
##########
@@ -62,78 +63,51 @@ TestPlan::~TestPlan() {
   for (auto& processor : configured_processors_) {
     processor->setScheduledState(core::ScheduledState::STOPPED);
   }
+  for (auto& connection : relationships_) {
+    // This is a patch solving circular references between processors and connections
+    connection->setSource(nullptr);
+    connection->setDestination(nullptr);
+  }
   controller_services_provider_->clearControllerServices();
 }
 
 std::shared_ptr<core::Processor> TestPlan::addProcessor(const std::shared_ptr<core::Processor> &processor, const std::string &name, const std::initializer_list<core::Relationship>& relationships,
-                                                        bool linkToPrevious) {
+    bool linkToPrevious) {
   if (finalized) {
     return nullptr;
   }
   std::lock_guard<std::recursive_mutex> guard(mutex);
-
   utils::Identifier uuid = utils::IdGenerator::getIdGenerator()->generate();
-
   processor->setStreamFactory(stream_factory);
   // initialize the processor
   processor->initialize();
   processor->setFlowIdentifier(flow_version_->getFlowIdentifier());
-
   processor_mapping_[processor->getUUID()] = processor;
-
   if (!linkToPrevious) {
     termination_ = *(relationships.begin());
   } else {
     std::shared_ptr<core::Processor> last = processor_queue_.back();
-
     if (last == nullptr) {
       last = processor;
       termination_ = *(relationships.begin());
     }
-
-    std::stringstream connection_name;
-    connection_name << last->getUUIDStr() << "-to-" << processor->getUUIDStr();
-    logger_->log_info("Creating %s connection for proc %d", connection_name.str(), processor_queue_.size() + 1);
-    std::shared_ptr<minifi::Connection> connection = std::make_shared<minifi::Connection>(flow_repo_, content_repo_, connection_name.str());
-
     for (const auto& relationship : relationships) {
-      connection->addRelationship(relationship);
-    }
-
-    // link the connections so that we can test results at the end for this
-    connection->setSource(last);
-    connection->setDestination(processor);
-
-    connection->setSourceUUID(last->getUUID());
-    connection->setDestinationUUID(processor->getUUID());
-    last->addConnection(connection);
-    if (last != processor) {
-      processor->addConnection(connection);
+      addConnection(last, relationship, processor);

Review comment:
       I am not sure I understand the question, but It is a prerequisite if you want to connect up multiple relationships separately.

##########
File path: libminifi/test/TestBase.cpp
##########
@@ -62,78 +63,51 @@ TestPlan::~TestPlan() {
   for (auto& processor : configured_processors_) {
     processor->setScheduledState(core::ScheduledState::STOPPED);
   }
+  for (auto& connection : relationships_) {
+    // This is a patch solving circular references between processors and connections
+    connection->setSource(nullptr);
+    connection->setDestination(nullptr);
+  }
   controller_services_provider_->clearControllerServices();
 }
 
 std::shared_ptr<core::Processor> TestPlan::addProcessor(const std::shared_ptr<core::Processor> &processor, const std::string &name, const std::initializer_list<core::Relationship>& relationships,
-                                                        bool linkToPrevious) {
+    bool linkToPrevious) {
   if (finalized) {
     return nullptr;
   }
   std::lock_guard<std::recursive_mutex> guard(mutex);
-
   utils::Identifier uuid = utils::IdGenerator::getIdGenerator()->generate();
-
   processor->setStreamFactory(stream_factory);
   // initialize the processor
   processor->initialize();
   processor->setFlowIdentifier(flow_version_->getFlowIdentifier());
-
   processor_mapping_[processor->getUUID()] = processor;
-
   if (!linkToPrevious) {
     termination_ = *(relationships.begin());
   } else {
     std::shared_ptr<core::Processor> last = processor_queue_.back();
-
     if (last == nullptr) {
       last = processor;
       termination_ = *(relationships.begin());
     }
-
-    std::stringstream connection_name;
-    connection_name << last->getUUIDStr() << "-to-" << processor->getUUIDStr();
-    logger_->log_info("Creating %s connection for proc %d", connection_name.str(), processor_queue_.size() + 1);
-    std::shared_ptr<minifi::Connection> connection = std::make_shared<minifi::Connection>(flow_repo_, content_repo_, connection_name.str());
-
     for (const auto& relationship : relationships) {
-      connection->addRelationship(relationship);
-    }
-
-    // link the connections so that we can test results at the end for this
-    connection->setSource(last);
-    connection->setDestination(processor);
-
-    connection->setSourceUUID(last->getUUID());
-    connection->setDestinationUUID(processor->getUUID());
-    last->addConnection(connection);
-    if (last != processor) {
-      processor->addConnection(connection);
+      addConnection(last, relationship, processor);

Review comment:
       I am not sure I understand the question, but is it not a prerequisite if you want to connect up multiple relationships separately?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553322983



##########
File path: extensions/librdkafka/tests/ConsumeKafkaTests.cpp
##########
@@ -0,0 +1,593 @@
+
+/**
+ * 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 "../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(), "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) {
+    reInitialize();
+
+    // Consumer chain
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+    std::shared_ptr<core::Processor> extract_text  = plan_->addProcessor("ExtractText", "extract_text", {success}, false);
+
+    // Set up connections
+    plan_->addConnection(consume_kafka, success, extract_text);
+    extract_text->setAutoTerminatedRelationships({success});
+
+    const auto bool_to_string = [] (const bool b) -> std::string { return b ? "true" : "false"; };
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::SecurityProtocol.getName(), security_protocol);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), topic_names);
+
+    optional_set_property(consume_kafka, ConsumeKafka::TopicNameFormat.getName(), topic_name_format);
+    optional_set_property(consume_kafka, ConsumeKafka::HonorTransactions.getName(), honor_transactions | utils::map(bool_to_string));
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+    optional_set_property(consume_kafka, ConsumeKafka::OffsetReset.getName(), offset_reset);
+    optional_set_property(consume_kafka, ConsumeKafka::KeyAttributeEncoding.getName(), key_attribute_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageDemarcator.getName(), message_demarcator);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageHeaderEncoding.getName(), message_header_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::HeadersToAddAsAttributes.getName(), headers_to_add_as_attributes);
+    optional_set_property(consume_kafka, ConsumeKafka::DuplicateHeaderHandling.getName(), duplicate_header_handling);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    plan_->setProperty(extract_text, ExtractText::Attribute.getName(), ATTRIBUTE_FOR_CAPTURING_CONTENT);
+
+    if (!expect_config_valid) {
+      const auto& message = messages_on_topic.front();

Review comment:
       Indeed. Removed.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553314598



##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);
+  }
+}
+
+void print_topics_list(std::shared_ptr<logging::Logger> logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger->log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld]",

Review comment:
       No, this used to be colored so that my logs pop out from the rdkafka api generated logs if I turn on debug=all, and I messed up with the deletion here :S




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553905938



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,569 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));
+  }
+  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::steady_clock::now();
+  auto elapsed = std::chrono::steady_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());
+    std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>
+      message { rd_kafka_consumer_poll(consumer_.get(), std::chrono::duration_cast<std::chrono::milliseconds>(max_poll_time_milliseconds_ - elapsed).count()), utils::rd_kafka_message_deleter() };
+    if (!message) {
+      break;
+    }
+    if (RD_KAFKA_RESP_ERR_NO_ERROR != message->err) {
+      logger_->log_error("Received message with error %d: %s", message->err, rd_kafka_err2str(message->err));
+      break;
+    }
+    utils::print_kafka_message(message.get(), logger_);
+    messages.emplace_back(std::move(message));
+    elapsed = std::chrono::steady_clock::now() - start;
+  }
+  return messages;
+}
+
+utils::KafkaEncoding ConsumeKafka::key_attr_encoding_attr_to_enum() const {
+  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() const {
+  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, "\"Message Header Encoding\" property not recognized.");
+}
+
+std::string ConsumeKafka::resolve_duplicate_headers(const std::vector<std::string>& matching_headers) const {
+  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) const {
+  // 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) const {
+  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_) {
+      ret = stream->write(data_,  gsl::narrow<int>(dataSize_));
+    }
+    return ret;
+  }
+};
+
+void ConsumeKafka::add_kafka_attributes_to_flowfile(std::shared_ptr<FlowFileRecord>& flow_file, const rd_kafka_message_t* message) const {
+  // 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, 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));
+}
+
+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) const {
+  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("Message received contains no data.");
+      continue;
+    }
+
+    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 { message_content };
+    if (message_demarcator_.size()) {
+      split_message = utils::StringUtils::split(message_content, message_demarcator_);
+    }

Review comment:
       Went with a ternary instead:
   ```c++
   std::vector<std::string> split_message{ message_demarcator_.size() ?
     utils::StringUtils::split(message_content, message_demarcator_) :
     std::vector<std::string>{ message_content }};
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553793221



##########
File path: libminifi/include/utils/ProcessorConfigUtils.h
##########
@@ -0,0 +1,80 @@
+/**
+ * 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) {
+  std::string value;
+  if (!context->getProperty(property_name, value)) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property missing or invalid");
+  }
+  return value;
+}
+
+std::vector<std::string> listFromCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  std::string property_string;
+  context->getProperty(property_name, property_string);
+  return utils::StringUtils::splitAndTrim(property_string, ",");
+}
+
+std::vector<std::string> listFromRequiredCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  return utils::StringUtils::splitAndTrim(getRequiredPropertyOrThrow(context, property_name), ",");
+}
+
+bool parseBooleanPropertyOrThrow(core::ProcessContext* context, const std::string& property_name) {
+  bool value;
+  std::string value_str = getRequiredPropertyOrThrow(context, property_name);
+  utils::optional<bool> maybe_value = utils::StringUtils::toBool(value_str);
+  if (!maybe_value) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property is invalid: value is " + value_str);

Review comment:
       Does it really matter though? I would be even happy to throw `std::runtime_error` here.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r528653994



##########
File path: libminifi/test/unit/StringUtilsTests.cpp
##########
@@ -50,6 +50,16 @@ TEST_CASE("TestStringUtils::split4", "[test split classname]") {
   REQUIRE(expected == StringUtils::split(org::apache::nifi::minifi::core::getClassName<org::apache::nifi::minifi::utils::StringUtils>(), "::"));
 }
 
+TEST_CASE("TestStringUtils::split5", "[test split delimiter not specified]") {

Review comment:
       This used to be a SPoF. The current behaviour is now to keep the input string if the delimiter is `""`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r577733272



##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,121 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error: " + error_msg);
+  }
+}
+
+void print_topics_list(logging::Logger& logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger.log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset: %" PRId64 ".",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+std::string get_human_readable_kafka_message_timestamp(const rd_kafka_message_t* rkmessage) {
+  rd_kafka_timestamp_type_t tstype;
+  int64_t timestamp;
+  timestamp = rd_kafka_message_timestamp(rkmessage, &tstype);
+  const char *tsname = "?";
+  if (tstype == RD_KAFKA_TIMESTAMP_CREATE_TIME) {
+    tsname = "create time";
+  } else if (tstype == RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME) {
+    tsname = "log append time";
+  }
+  const int64_t seconds_since_timestamp = timestamp == -1 ? 0 : static_cast<int64_t>(time(NULL)) - static_cast<int64_t>(timestamp / 1000);
+  return {"[Timestamp](" + std::string(tsname) + " " + std::to_string(timestamp) + " (" + std::to_string(seconds_since_timestamp) + " s ago)"};
+}
+
+std::string get_human_readable_kafka_message_headers(const rd_kafka_message_t* rkmessage, logging::Logger& logger) {
+  rd_kafka_headers_t* hdrs;
+  const rd_kafka_resp_err_t get_header_response = rd_kafka_message_headers(rkmessage, &hdrs);
+  if (RD_KAFKA_RESP_ERR_NO_ERROR == get_header_response) {
+    std::vector<std::string> header_list;
+    kafka_headers_for_each(hdrs, [&] (const std::string& key, gsl::span<const char> val) { header_list.emplace_back(key + ": " + std::string{ val.data(), val.size() }); });
+    return StringUtils::join(", ", header_list);
+  }
+  if (RD_KAFKA_RESP_ERR__NOENT == get_header_response) {
+    return "[None]";
+  }
+  logger.log_error("Failed to fetch message headers: %d: %s", rd_kafka_last_error(), rd_kafka_err2str(rd_kafka_last_error()));
+  return "[Error]";
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, logging::Logger& logger) {

Review comment:
       This is still taking an unchecked pointer

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,121 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error: " + error_msg);
+  }
+}
+
+void print_topics_list(logging::Logger& logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger.log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset: %" PRId64 ".",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+std::string get_human_readable_kafka_message_timestamp(const rd_kafka_message_t* rkmessage) {
+  rd_kafka_timestamp_type_t tstype;
+  int64_t timestamp;
+  timestamp = rd_kafka_message_timestamp(rkmessage, &tstype);
+  const char *tsname = "?";
+  if (tstype == RD_KAFKA_TIMESTAMP_CREATE_TIME) {
+    tsname = "create time";
+  } else if (tstype == RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME) {
+    tsname = "log append time";
+  }
+  const int64_t seconds_since_timestamp = timestamp == -1 ? 0 : static_cast<int64_t>(time(NULL)) - static_cast<int64_t>(timestamp / 1000);
+  return {"[Timestamp](" + std::string(tsname) + " " + std::to_string(timestamp) + " (" + std::to_string(seconds_since_timestamp) + " s ago)"};
+}
+
+std::string get_human_readable_kafka_message_headers(const rd_kafka_message_t* rkmessage, logging::Logger& logger) {
+  rd_kafka_headers_t* hdrs;
+  const rd_kafka_resp_err_t get_header_response = rd_kafka_message_headers(rkmessage, &hdrs);
+  if (RD_KAFKA_RESP_ERR_NO_ERROR == get_header_response) {
+    std::vector<std::string> header_list;
+    kafka_headers_for_each(hdrs, [&] (const std::string& key, gsl::span<const char> val) { header_list.emplace_back(key + ": " + std::string{ val.data(), val.size() }); });
+    return StringUtils::join(", ", header_list);
+  }
+  if (RD_KAFKA_RESP_ERR__NOENT == get_header_response) {
+    return "[None]";
+  }
+  logger.log_error("Failed to fetch message headers: %d: %s", rd_kafka_last_error(), rd_kafka_err2str(rd_kafka_last_error()));
+  return "[Error]";
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, logging::Logger& logger) {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    const std::string error_msg = "ConsumeKafka: received error message from broker. Librdkafka error msg: " + std::string(rd_kafka_err2str(rkmessage->err));
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, error_msg);
+  }
+  std::string topicName = rd_kafka_topic_name(rkmessage->rkt);
+  std::string message(reinterpret_cast<char*>(rkmessage->payload), rkmessage->len);
+  const char* key = reinterpret_cast<const char*>(rkmessage->key);
+  const std::size_t key_len = rkmessage->key_len;
+
+  std::string message_as_string;
+  message_as_string += "[Topic](" + topicName + "), ";
+  message_as_string += "[Key](" + (key != nullptr ? std::string(key, key_len) : std::string("[None]")) + "), ";
+  message_as_string += "[Offset](" +  std::to_string(rkmessage->offset) + "), ";
+  message_as_string += "[Message Length](" + std::to_string(rkmessage->len) + "), ";
+  message_as_string += get_human_readable_kafka_message_timestamp(rkmessage) + "), ";
+  message_as_string += "[Headers](";
+  message_as_string += get_human_readable_kafka_message_headers(rkmessage, logger) + ")";
+  message_as_string += "[Payload](" + message + ")";
+
+  logger.log_debug("Message: %s", message_as_string.c_str());
+}
+
+std::string get_encoded_string(const std::string& input, KafkaEncoding encoding) {
+  switch (encoding) {
+    case KafkaEncoding::UTF8:
+      return input;
+    case KafkaEncoding::HEX:
+      return StringUtils::to_hex(input, /* uppercase = */ true);
+  }
+  throw std::runtime_error("Invalid encoding selected: " + input);
+}
+
+optional<std::string> get_encoded_message_key(const rd_kafka_message_t* message, KafkaEncoding encoding) {

Review comment:
       This is still taking an unchecked pointer

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,121 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error: " + error_msg);
+  }
+}
+
+void print_topics_list(logging::Logger& logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {

Review comment:
       This is still taking an unchecked pointer

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,121 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {

Review comment:
       This is still taking an unchecked pointer

##########
File path: extensions/librdkafka/rdkafka_utils.h
##########
@@ -0,0 +1,105 @@
+/**
+ * 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 "utils/gsl.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;
+    // }
+    rd_kafka_destroy(ptr);
+  }
+};
+
+struct rd_kafka_consumer_deleter {
+  void operator()(rd_kafka_t* ptr) const noexcept {
+    rd_kafka_consumer_close(ptr);
+    rd_kafka_destroy(ptr);
+  }
+};
+
+struct rd_kafka_topic_partition_list_deleter {
+  void operator()(rd_kafka_topic_partition_list_t* ptr) const noexcept { rd_kafka_topic_partition_list_destroy(ptr); }
+};
+
+struct rd_kafka_topic_conf_deleter {
+  void operator()(rd_kafka_topic_conf_t* ptr) const noexcept { rd_kafka_topic_conf_destroy(ptr); }
+};
+struct rd_kafka_topic_deleter {
+  void operator()(rd_kafka_topic_t* ptr) const noexcept { rd_kafka_topic_destroy(ptr); }
+};
+
+struct rd_kafka_message_deleter {
+  void operator()(rd_kafka_message_t* ptr) const noexcept { rd_kafka_message_destroy(ptr); }
+};
+
+struct rd_kafka_headers_deleter {
+  void operator()(rd_kafka_headers_t* ptr) const noexcept { rd_kafka_headers_destroy(ptr); }
+};
+
+template <typename T>
+void kafka_headers_for_each(const rd_kafka_headers_t* headers, T key_value_handle) {

Review comment:
       This is still taking an unchecked pointer

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,121 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error: " + error_msg);
+  }
+}
+
+void print_topics_list(logging::Logger& logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger.log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset: %" PRId64 ".",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+std::string get_human_readable_kafka_message_timestamp(const rd_kafka_message_t* rkmessage) {

Review comment:
       This is still taking an unchecked pointer

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,121 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error: " + error_msg);
+  }
+}
+
+void print_topics_list(logging::Logger& logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger.log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset: %" PRId64 ".",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+std::string get_human_readable_kafka_message_timestamp(const rd_kafka_message_t* rkmessage) {
+  rd_kafka_timestamp_type_t tstype;
+  int64_t timestamp;
+  timestamp = rd_kafka_message_timestamp(rkmessage, &tstype);
+  const char *tsname = "?";
+  if (tstype == RD_KAFKA_TIMESTAMP_CREATE_TIME) {
+    tsname = "create time";
+  } else if (tstype == RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME) {
+    tsname = "log append time";
+  }
+  const int64_t seconds_since_timestamp = timestamp == -1 ? 0 : static_cast<int64_t>(time(NULL)) - static_cast<int64_t>(timestamp / 1000);
+  return {"[Timestamp](" + std::string(tsname) + " " + std::to_string(timestamp) + " (" + std::to_string(seconds_since_timestamp) + " s ago)"};
+}
+
+std::string get_human_readable_kafka_message_headers(const rd_kafka_message_t* rkmessage, logging::Logger& logger) {

Review comment:
       This is still taking an unchecked pointer

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,570 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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 comma separated list to match against all message headers. Any message header whose name matches an item from the list will be added to the FlowFile "
+      "as an Attribute. If not specified, no Header values will be added as FlowFile attributes. The behaviour on when multiple headers of the same name are present is set using "
+      "the DuplicateHeaderHandling attribute.")
+  ->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 */) {
+  gsl_Expects(context);
+  // 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);
+}
+
+namespace {
+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:");
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Failed to create Kafka consumer " + error_msg);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));
+  }
+  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::steady_clock::now();
+  auto elapsed = std::chrono::steady_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());
+    std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>
+      message { rd_kafka_consumer_poll(consumer_.get(), std::chrono::duration_cast<std::chrono::milliseconds>(max_poll_time_milliseconds_ - elapsed).count()), utils::rd_kafka_message_deleter() };
+    if (!message) {
+      break;
+    }
+    if (RD_KAFKA_RESP_ERR_NO_ERROR != message->err) {
+      logger_->log_error("Received message with error %d: %s", message->err, rd_kafka_err2str(message->err));
+      break;
+    }
+    utils::print_kafka_message(message.get(), *logger_);
+    messages.emplace_back(std::move(message));
+    elapsed = std::chrono::steady_clock::now() - start;
+  }
+  return messages;
+}
+
+utils::KafkaEncoding ConsumeKafka::key_attr_encoding_attr_to_enum() const {
+  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() const {
+  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, "\"Message Header Encoding\" property not recognized.");
+}
+
+std::string ConsumeKafka::resolve_duplicate_headers(const std::vector<std::string>& matching_headers) const {
+  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) const {

Review comment:
       This is still taking an unchecked pointer

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,570 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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 comma separated list to match against all message headers. Any message header whose name matches an item from the list will be added to the FlowFile "
+      "as an Attribute. If not specified, no Header values will be added as FlowFile attributes. The behaviour on when multiple headers of the same name are present is set using "
+      "the DuplicateHeaderHandling attribute.")
+  ->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 */) {
+  gsl_Expects(context);
+  // 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);
+}
+
+namespace {
+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:");
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Failed to create Kafka consumer " + error_msg);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));
+  }
+  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::steady_clock::now();
+  auto elapsed = std::chrono::steady_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());
+    std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>
+      message { rd_kafka_consumer_poll(consumer_.get(), std::chrono::duration_cast<std::chrono::milliseconds>(max_poll_time_milliseconds_ - elapsed).count()), utils::rd_kafka_message_deleter() };
+    if (!message) {
+      break;
+    }
+    if (RD_KAFKA_RESP_ERR_NO_ERROR != message->err) {
+      logger_->log_error("Received message with error %d: %s", message->err, rd_kafka_err2str(message->err));
+      break;
+    }
+    utils::print_kafka_message(message.get(), *logger_);
+    messages.emplace_back(std::move(message));
+    elapsed = std::chrono::steady_clock::now() - start;
+  }
+  return messages;
+}
+
+utils::KafkaEncoding ConsumeKafka::key_attr_encoding_attr_to_enum() const {
+  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() const {
+  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, "\"Message Header Encoding\" property not recognized.");
+}
+
+std::string ConsumeKafka::resolve_duplicate_headers(const std::vector<std::string>& matching_headers) const {
+  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) const {
+  // 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 < 200) {
+      logger_->log_debug("%.*s", static_cast<int>(size), value);
+    } else {
+      logger_->log_debug("%.*s...", 200, 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) const {

Review comment:
       This is still taking an unchecked pointer

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,570 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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 comma separated list to match against all message headers. Any message header whose name matches an item from the list will be added to the FlowFile "
+      "as an Attribute. If not specified, no Header values will be added as FlowFile attributes. The behaviour on when multiple headers of the same name are present is set using "
+      "the DuplicateHeaderHandling attribute.")
+  ->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 */) {
+  gsl_Expects(context);
+  // 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);
+}
+
+namespace {
+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:");
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Failed to create Kafka consumer " + error_msg);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {

Review comment:
       This is still taking an unchecked pointer

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,570 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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 comma separated list to match against all message headers. Any message header whose name matches an item from the list will be added to the FlowFile "
+      "as an Attribute. If not specified, no Header values will be added as FlowFile attributes. The behaviour on when multiple headers of the same name are present is set using "
+      "the DuplicateHeaderHandling attribute.")
+  ->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 */) {
+  gsl_Expects(context);
+  // 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);
+}
+
+namespace {
+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:");
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Failed to create Kafka consumer " + error_msg);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));
+  }
+  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::steady_clock::now();
+  auto elapsed = std::chrono::steady_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());
+    std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>
+      message { rd_kafka_consumer_poll(consumer_.get(), std::chrono::duration_cast<std::chrono::milliseconds>(max_poll_time_milliseconds_ - elapsed).count()), utils::rd_kafka_message_deleter() };
+    if (!message) {
+      break;
+    }
+    if (RD_KAFKA_RESP_ERR_NO_ERROR != message->err) {
+      logger_->log_error("Received message with error %d: %s", message->err, rd_kafka_err2str(message->err));
+      break;
+    }
+    utils::print_kafka_message(message.get(), *logger_);
+    messages.emplace_back(std::move(message));
+    elapsed = std::chrono::steady_clock::now() - start;
+  }
+  return messages;
+}
+
+utils::KafkaEncoding ConsumeKafka::key_attr_encoding_attr_to_enum() const {
+  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() const {
+  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, "\"Message Header Encoding\" property not recognized.");
+}
+
+std::string ConsumeKafka::resolve_duplicate_headers(const std::vector<std::string>& matching_headers) const {
+  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) const {
+  // 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 < 200) {
+      logger_->log_debug("%.*s", static_cast<int>(size), value);
+    } else {
+      logger_->log_debug("%.*s...", 200, 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) const {
+  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;
+}
+
+void ConsumeKafka::add_kafka_attributes_to_flowfile(std::shared_ptr<FlowFileRecord>& flow_file, const rd_kafka_message_t* message) const {

Review comment:
       This is still taking unchecked pointers




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r538358865



##########
File path: libminifi/test/TestBase.h
##########
@@ -200,7 +199,7 @@ class LogTestController {
     }
     my_properties_->set("logger.root", "ERROR,ostream");
     my_properties_->set("logger." + core::getClassName<LogTestController>(), "INFO");
-    my_properties_->set("logger." + core::getClassName<logging::LoggerConfiguration>(), "DEBUG");
+    my_properties_->set("logger." + core::getClassName<logging::LoggerConfiguration>(), "INFO");

Review comment:
       This breaks C2JstackTest. Once  #953 is merged, rebasing on it should fix this issue. 
   *Update*: It is no longer an issue.

##########
File path: libminifi/test/TestBase.h
##########
@@ -200,7 +199,7 @@ class LogTestController {
     }
     my_properties_->set("logger.root", "ERROR,ostream");
     my_properties_->set("logger." + core::getClassName<LogTestController>(), "INFO");
-    my_properties_->set("logger." + core::getClassName<logging::LoggerConfiguration>(), "DEBUG");
+    my_properties_->set("logger." + core::getClassName<logging::LoggerConfiguration>(), "INFO");

Review comment:
       This breaks C2JstackTest. Once  #953 is merged, rebasing on it should fix this issue. 
   **Update**: It is no longer an issue.

##########
File path: libminifi/test/TestBase.h
##########
@@ -200,7 +199,7 @@ class LogTestController {
     }
     my_properties_->set("logger.root", "ERROR,ostream");
     my_properties_->set("logger." + core::getClassName<LogTestController>(), "INFO");
-    my_properties_->set("logger." + core::getClassName<logging::LoggerConfiguration>(), "DEBUG");
+    my_properties_->set("logger." + core::getClassName<logging::LoggerConfiguration>(), "INFO");

Review comment:
       It is no longer an issue.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r560854446



##########
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;
+    // }
+    rd_kafka_destroy(ptr);
+  }
+};
+
+struct rd_kafka_consumer_deleter {
+  void operator()(rd_kafka_t* ptr) const noexcept {
+    rd_kafka_consumer_close(ptr);
+    rd_kafka_destroy(ptr);
+  }
+};
+
+struct rd_kafka_topic_partition_list_deleter {
+  void operator()(rd_kafka_topic_partition_list_t* ptr) const noexcept { rd_kafka_topic_partition_list_destroy(ptr); }
+};
+
+struct rd_kafka_topic_conf_deleter {
+  void operator()(rd_kafka_topic_conf_t* ptr) const noexcept { rd_kafka_topic_conf_destroy(ptr); }
+};
+struct rd_kafka_topic_deleter {
+  void operator()(rd_kafka_topic_t* ptr) const noexcept { rd_kafka_topic_destroy(ptr); }
+};
+
+struct rd_kafka_message_deleter {
+  void operator()(rd_kafka_message_t* ptr) const noexcept { rd_kafka_message_destroy(ptr); }
+};
+
+struct rd_kafka_headers_deleter {
+  void operator()(rd_kafka_headers_t* ptr) const noexcept { rd_kafka_headers_destroy(ptr); }
+};
+
+template <typename T>
+void kafka_headers_for_each(const rd_kafka_headers_t* headers, T key_value_handle) {
+  const char *key;  // Null terminated, not to be freed
+  const void *value;
+  std::size_t size;
+  for (std::size_t i = 0; RD_KAFKA_RESP_ERR_NO_ERROR == rd_kafka_header_get_all(headers, i, &key, &value, &size); ++i) {
+    key_value_handle(std::string(key), std::string(static_cast<const char*>(value), size));

Review comment:
       It is a `void*` data type. I am not exactly sure if `0` termination is guaranteed here.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r577708270



##########
File path: extensions/librdkafka/ConsumeKafka.h
##########
@@ -124,6 +124,17 @@ class ConsumeKafka : public core::Processor {
   // Initialize, overwrite by NiFi RetryFlowFile
   void initialize() override;
 
+  class WriteCallback : public OutputStreamCallback {

Review comment:
       Indeed it does not need to be, this is just where I found it on the first match when searching similar patterns (ApplyTemplate). Changed it to `private`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553321849



##########
File path: extensions/librdkafka/tests/ConsumeKafkaTests.cpp
##########
@@ -0,0 +1,593 @@
+
+/**
+ * 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 "../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(), "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

Review comment:
       Yes, updated.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r567646819



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,579 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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 */) {

Review comment:
       Is it the responsibility of the processor to ensure it is run in a valid context?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r577510561



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,578 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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 comma separated list to match against all message headers. Any message header whose name matches an item from the list will be added to the FlowFile "
+      "as an Attribute. If not specified, no Header values will be added as FlowFile attributes. The behaviour on when multiple headers of the same name are present is set using "
+      "the DuplicateHeaderHandling attribute.")
+  ->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 */) {
+  gsl_Expects(context);
+  // 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);
+}
+
+namespace {
+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");
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Failed to create Kafka consumer " + error_msg);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));
+  }
+  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::steady_clock::now();
+  auto elapsed = std::chrono::steady_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());
+    std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>
+      message { rd_kafka_consumer_poll(consumer_.get(), std::chrono::duration_cast<std::chrono::milliseconds>(max_poll_time_milliseconds_ - elapsed).count()), utils::rd_kafka_message_deleter() };
+    if (!message) {
+      break;
+    }
+    if (RD_KAFKA_RESP_ERR_NO_ERROR != message->err) {
+      logger_->log_error("Received message with error %d: %s", message->err, rd_kafka_err2str(message->err));
+      break;
+    }
+    utils::print_kafka_message(message.get(), *logger_);
+    messages.emplace_back(std::move(message));
+    elapsed = std::chrono::steady_clock::now() - start;
+  }
+  return messages;
+}
+
+utils::KafkaEncoding ConsumeKafka::key_attr_encoding_attr_to_enum() const {
+  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() const {
+  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, "\"Message Header Encoding\" property not recognized.");
+}
+
+std::string ConsumeKafka::resolve_duplicate_headers(const std::vector<std::string>& matching_headers) const {
+  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) const {
+  // 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 < 200) {
+      logger_->log_debug("%.*s", static_cast<int>(size), value);
+    } else {
+      logger_->log_debug("%.*s...", 200, 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) const {
+  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_) {
+      ret = stream->write(data_,  gsl::narrow<int>(dataSize_));
+    }
+    return ret;
+  }
+};
+
+void ConsumeKafka::add_kafka_attributes_to_flowfile(std::shared_ptr<FlowFileRecord>& flow_file, const rd_kafka_message_t* message) const {
+  // 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, 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));
+}
+
+utils::optional<std::vector<std::shared_ptr<FlowFileRecord>>> ConsumeKafka::transform_pending_messages_into_flowfiles(core::ProcessSession* session) const {

Review comment:
       My reasoning on passing pointers for these functions was that this is how the processor is get called as well, so this was the simplest way to forward the args. I assumed that by the time the onTrigger is called on the processors in a flow the session is already checked for validity.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553788782



##########
File path: libminifi/src/utils/StringUtils.cpp
##########
@@ -46,8 +46,13 @@ std::string StringUtils::trim(const std::string& s) {
   return trimRight(trimLeft(s));
 }
 
-std::vector<std::string> StringUtils::split(const std::string &str, const std::string &delimiter) {
+template<typename Fun>
+std::vector<std::string> split_transformed(const std::string& str, const std::string& delimiter, Fun transformation) {
   std::vector<std::string> result;
+  if (delimiter.empty()) {
+    std::transform(str.begin(), str.end(), std::back_inserter(result), [] (const char c) { return std::string{c}; });

Review comment:
       Updated.

##########
File path: libminifi/src/utils/StringUtils.cpp
##########
@@ -46,8 +46,13 @@ std::string StringUtils::trim(const std::string& s) {
   return trimRight(trimLeft(s));
 }
 
-std::vector<std::string> StringUtils::split(const std::string &str, const std::string &delimiter) {
+template<typename Fun>
+std::vector<std::string> split_transformed(const std::string& str, const std::string& delimiter, Fun transformation) {
   std::vector<std::string> result;
+  if (delimiter.empty()) {
+    std::transform(str.begin(), str.end(), std::back_inserter(result), [] (const char c) { return std::string{c}; });

Review comment:
       Fixed.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r577491948



##########
File path: extensions/librdkafka/ConsumeKafka.h
##########
@@ -0,0 +1,181 @@
+/**
+ * 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 <memory>
+#include <string>
+#include <utility>
+#include <vector>
+
+#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 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()) :

Review comment:
       Agreed, I did not even notice it when copy-pasting another processor. Maybe we should consider updating all occurences of this usage?
   
   ![image](https://user-images.githubusercontent.com/64011968/108190395-2dff7b00-7112-11eb-856b-e0da0aab39bd.png)
   




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r576730170



##########
File path: libminifi/test/TestBase.cpp
##########
@@ -111,28 +109,19 @@ std::shared_ptr<core::Processor> TestPlan::addProcessor(const std::shared_ptr<co
     }
     relationships_.push_back(connection);
   }
-
   std::shared_ptr<core::ProcessorNode> node = std::make_shared<core::ProcessorNode>(processor);
-
   processor_nodes_.push_back(node);
-
   // std::shared_ptr<core::ProcessContext> context = std::make_shared<core::ProcessContext>(node, controller_services_provider_, prov_repo_, flow_repo_, configuration_, content_repo_);
-
   auto contextBuilder = core::ClassLoader::getDefaultClassLoader().instantiate<core::ProcessContextBuilder>("ProcessContextBuilder");
-
   contextBuilder = contextBuilder->withContentRepository(content_repo_)->withFlowFileRepository(flow_repo_)->withProvider(controller_services_provider_.get())->withProvenanceRepository(prov_repo_)->withConfiguration(configuration_);
-
   auto context = contextBuilder->build(node);
-
   processor_contexts_.push_back(context);
-
   processor_queue_.push_back(processor);
-
   return processor;
 }
 
 std::shared_ptr<core::Processor> TestPlan::addProcessor(const std::string &processor_name, const utils::Identifier& uuid, const std::string &name,
-                                                        const std::initializer_list<core::Relationship>& relationships, bool linkToPrevious) {
+  const std::initializer_list<core::Relationship>& relationships, bool linkToPrevious) {

Review comment:
       Continuation indentation should be 4 spaces or aligned. My preference is 4 spaces, the style guide's is aligned.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r576122753



##########
File path: libminifi/test/TestBase.cpp
##########
@@ -62,78 +63,51 @@ TestPlan::~TestPlan() {
   for (auto& processor : configured_processors_) {
     processor->setScheduledState(core::ScheduledState::STOPPED);
   }
+  for (auto& connection : relationships_) {
+    // This is a patch solving circular references between processors and connections
+    connection->setSource(nullptr);
+    connection->setDestination(nullptr);
+  }
   controller_services_provider_->clearControllerServices();
 }
 
 std::shared_ptr<core::Processor> TestPlan::addProcessor(const std::shared_ptr<core::Processor> &processor, const std::string &name, const std::initializer_list<core::Relationship>& relationships,
-                                                        bool linkToPrevious) {
+    bool linkToPrevious) {
   if (finalized) {
     return nullptr;
   }
   std::lock_guard<std::recursive_mutex> guard(mutex);
-
   utils::Identifier uuid = utils::IdGenerator::getIdGenerator()->generate();
-
   processor->setStreamFactory(stream_factory);
   // initialize the processor
   processor->initialize();
   processor->setFlowIdentifier(flow_version_->getFlowIdentifier());
-
   processor_mapping_[processor->getUUID()] = processor;
-
   if (!linkToPrevious) {
     termination_ = *(relationships.begin());
   } else {
     std::shared_ptr<core::Processor> last = processor_queue_.back();
-
     if (last == nullptr) {
       last = processor;
       termination_ = *(relationships.begin());
     }
-
-    std::stringstream connection_name;
-    connection_name << last->getUUIDStr() << "-to-" << processor->getUUIDStr();
-    logger_->log_info("Creating %s connection for proc %d", connection_name.str(), processor_queue_.size() + 1);
-    std::shared_ptr<minifi::Connection> connection = std::make_shared<minifi::Connection>(flow_repo_, content_repo_, connection_name.str());
-
     for (const auto& relationship : relationships) {
-      connection->addRelationship(relationship);
-    }
-
-    // link the connections so that we can test results at the end for this
-    connection->setSource(last);
-    connection->setDestination(processor);
-
-    connection->setSourceUUID(last->getUUID());
-    connection->setDestinationUUID(processor->getUUID());
-    last->addConnection(connection);
-    if (last != processor) {
-      processor->addConnection(connection);
+      addConnection(last, relationship, processor);

Review comment:
       Why do you expect behavioural differences between the two? If you want to have more ellaborate test-setups, you need to have the flexibility of separately linking up relationships on processors.
   
   From a more pragmatical standpoint, this change is also tested through all the unit tests ran.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] fgerlits commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
fgerlits commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553410984



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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());

Review comment:
       Please update PROCESSORS.md, as well.  (Required properties should be in bold.)




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553877982



##########
File path: libminifi/include/utils/ProcessorConfigUtils.h
##########
@@ -0,0 +1,80 @@
+/**
+ * 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) {
+  std::string value;
+  if (!context->getProperty(property_name, value)) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property missing or invalid");
+  }
+  return value;
+}
+
+std::vector<std::string> listFromCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  std::string property_string;
+  context->getProperty(property_name, property_string);
+  return utils::StringUtils::splitAndTrim(property_string, ",");
+}
+
+std::vector<std::string> listFromRequiredCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  return utils::StringUtils::splitAndTrim(getRequiredPropertyOrThrow(context, property_name), ",");
+}
+
+bool parseBooleanPropertyOrThrow(core::ProcessContext* context, const std::string& property_name) {
+  bool value;
+  std::string value_str = getRequiredPropertyOrThrow(context, property_name);
+  utils::optional<bool> maybe_value = utils::StringUtils::toBool(value_str);
+  if (!maybe_value) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property is invalid: value is " + value_str);

Review comment:
       Updated to `runtime_error`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r560990362



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,579 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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.")

Review comment:
       How are messages bundled together into a flow file? I thought we emit one flow file per message.

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,579 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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 */) {

Review comment:
       I would consider checking that `context` is valid and throwing if not. Alternatively, setting a precondition with `gsl_Expects(context)` is fine too.

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,579 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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) {

Review comment:
       
   
   This could take a reference instead of a pointer. Currently its signature allows for null but it crashes on null.
   

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,579 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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:
       This could take a reference instead of a pointer. Currently its signature allows for null but it crashes on null.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553170430



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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*/) {

Review comment:
       Updated.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] fgerlits commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
fgerlits commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553426694



##########
File path: extensions/librdkafka/docker_tests/CMakeLists.txt
##########
@@ -0,0 +1,36 @@
+#

Review comment:
       > These are manual tests that are ran for the processor. I would not want a PR with a processor that does not involve the testing done for it.
   
   But I can't see any docker tests in this PR.  We have a CMakeLists file but the list of source files it applies to is empty.

##########
File path: extensions/librdkafka/tests/ConsumeKafkaTests.cpp
##########
@@ -0,0 +1,593 @@
+
+/**
+ * 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 "../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(), "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) {
+    reInitialize();
+
+    // Consumer chain
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+    std::shared_ptr<core::Processor> extract_text  = plan_->addProcessor("ExtractText", "extract_text", {success}, false);
+
+    // Set up connections
+    plan_->addConnection(consume_kafka, success, extract_text);
+    extract_text->setAutoTerminatedRelationships({success});
+
+    const auto bool_to_string = [] (const bool b) -> std::string { return b ? "true" : "false"; };
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::SecurityProtocol.getName(), security_protocol);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), topic_names);
+
+    optional_set_property(consume_kafka, ConsumeKafka::TopicNameFormat.getName(), topic_name_format);
+    optional_set_property(consume_kafka, ConsumeKafka::HonorTransactions.getName(), honor_transactions | utils::map(bool_to_string));
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+    optional_set_property(consume_kafka, ConsumeKafka::OffsetReset.getName(), offset_reset);
+    optional_set_property(consume_kafka, ConsumeKafka::KeyAttributeEncoding.getName(), key_attribute_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageDemarcator.getName(), message_demarcator);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageHeaderEncoding.getName(), message_header_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::HeadersToAddAsAttributes.getName(), headers_to_add_as_attributes);
+    optional_set_property(consume_kafka, ConsumeKafka::DuplicateHeaderHandling.getName(), duplicate_header_handling);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    plan_->setProperty(extract_text, ExtractText::Attribute.getName(), ATTRIBUTE_FOR_CAPTURING_CONTENT);
+
+    if (!expect_config_valid) {
+      const auto& message = messages_on_topic.front();
+      REQUIRE_THROWS(plan_->scheduleProcessor(consume_kafka));
+      return;
+    } else {
+      plan_->scheduleProcessors();
+    }
+
+    std::unique_ptr<rd_kafka_conf_t, utils::rd_kafka_conf_deleter> conf_;
+    std::unique_ptr<rd_kafka_t, utils::rd_kafka_consumer_deleter> consumer_;
+
+    const bool is_transactional = std::count(transaction_events.cbegin(), transaction_events.cend(), KafkaTestProducer::PublishEvent::TRANSACTION_START);
+    const bool transactions_committed = transaction_events.back() == KafkaTestProducer::PublishEvent::TRANSACTION_COMMIT;
+
+    KafkaTestProducer producer(kafka_brokers, PRODUCER_TOPIC, is_transactional);
+    producer.publish_messages_to_topic(messages_on_topic, TEST_MESSAGE_KEY, transaction_events, message_headers, message_header_encoding);
+
+
+    std::vector<std::shared_ptr<core::FlowFile>> flow_files_produced;
+    for (std::size_t num_expected_messages_processed = 0; num_expected_messages_processed < messages_on_topic.size(); num_expected_messages_processed += std::stoi(max_poll_records.value_or("1"))) {
+      plan_->increment_location();
+      if ((honor_transactions && false == honor_transactions.value()) || (is_transactional && !transactions_committed)) {
+        INFO("Non-committed messages received.");
+        REQUIRE(false == plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+        return;
+      }
+      {
+        SCOPED_INFO("ConsumeKafka timed out when waiting to receive the message published to the kafka broker.");
+        REQUIRE(plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+      }
+      std::size_t num_flow_files_produced = plan_->getNumFlowFileProducedByCurrentProcessor();
+      plan_->increment_location();
+      for (std::size_t times_extract_text_run = 0; times_extract_text_run < num_flow_files_produced; ++times_extract_text_run) {
+        plan_->runCurrentProcessor();  // ExtractText
+        std::shared_ptr<core::FlowFile> flow_file = plan_->getFlowFileProducedByCurrentProcessor();
+        for (const auto& exp_header : expect_header_attributes) {
+          SCOPED_INFO("ConsumeKafka did not produce the expected flowfile attribute from message header: " << exp_header.first << ".");
+          const auto header_attr_opt = flow_file->getAttribute(exp_header.first);
+          REQUIRE(header_attr_opt);
+          REQUIRE(exp_header.second == header_attr_opt.value().get());
+        }
+        {
+          SCOPED_INFO("Message key is missing or incorrect (potential encoding mismatch).");
+          REQUIRE(TEST_MESSAGE_KEY == decode_key(flow_file->getAttribute(ConsumeKafka::KAFKA_MESSAGE_KEY_ATTR).value().get(), key_attribute_encoding));
+          REQUIRE("1" == flow_file->getAttribute(ConsumeKafka::KAFKA_COUNT_ATTR).value().get());
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_OFFSET_ATTR));
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_PARTITION_ATTR));
+          REQUIRE(PRODUCER_TOPIC == flow_file->getAttribute(ConsumeKafka::KAFKA_TOPIC_ATTR).value().get());
+        }
+        flow_files_produced.emplace_back(std::move(flow_file));
+      }
+      plan_->reset_location();
+    }
+
+    const auto contentOrderOfFlowFile = [&] (const std::shared_ptr<core::FlowFile>& lhs, const std::shared_ptr<core::FlowFile>& rhs) {
+      return lhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() < rhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get();
+    };
+    {
+      SCOPED_INFO("The flowfiles generated by ConsumeKafka are invalid (probably nullptr).");
+      CHECK_NOTHROW(std::sort(flow_files_produced.begin(), flow_files_produced.end(), contentOrderOfFlowFile));
+    }
+    std::vector<std::string> sorted_split_messages = sort_and_split_messages(messages_on_topic, message_demarcator);
+    const auto flow_file_content_matches_message = [&] (const std::shared_ptr<core::FlowFile>& flowfile, const std::string message) {
+      return flowfile->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() == message;
+    };
+
+    logger_->log_debug("************");
+    std::string expected = "Expected: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      expected += sorted_split_messages[i] + ", ";
+    }
+    std::string   actual = "  Actual: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      actual += flow_files_produced[i]->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() + ", ";
+    }
+    logger_->log_debug("%s", expected.c_str());
+    logger_->log_debug("%s", actual.c_str());
+    logger_->log_debug("************");
+
+    INFO("The messages received by ConsumeKafka do not match those published");
+    REQUIRE(std::equal(flow_files_produced.begin(), flow_files_produced.end(), sorted_split_messages.begin(), flow_file_content_matches_message));
+  }
+};
+
+class ConsumeKafkaContinuousPublishingTest : public ConsumeKafkaTest {
+ public:
+  ConsumeKafkaContinuousPublishingTest() : ConsumeKafkaTest() {}
+  virtual ~ConsumeKafkaContinuousPublishingTest() {
+    logTestController_.reset();
+  }
+
+  void single_consumer_with_continuous_message_producing(
+      const uint64_t msg_periodicity_ms,
+      const std::string& kafka_brokers,
+      const optional<std::string>& group_id,
+      const optional<std::string>& max_poll_records,
+      const optional<std::string>& max_poll_time,
+      const optional<std::string>& session_timeout) {
+    reInitialize();
+
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+
+    plan_->setProperty(consume_kafka, "allow.auto.create.topics", "true", true);  // Seems like the topic tests work without this
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), PRODUCER_TOPIC);
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    consume_kafka->setAutoTerminatedRelationships({success});
+
+    KafkaTestProducer producer("localhost:9092", PRODUCER_TOPIC, /* transactional = */ false);
+
+    std::atomic_bool producer_loop_stop{ false };
+    std::size_t num_messages_sent = 0;  // When on C++14 make this an initialized lambda capture
+    const auto producer_loop = [&] {
+      std::this_thread::sleep_for(std::chrono::milliseconds(100));
+      while (!producer_loop_stop) {
+        producer.publish_messages_to_topic({ "Message after " + std::to_string(msg_periodicity_ms * num_messages_sent) + " ms"}, TEST_MESSAGE_KEY, { PUBLISH }, {}, {});
+        ++num_messages_sent;
+        std::this_thread::sleep_for(std::chrono::milliseconds(msg_periodicity_ms));
+      }
+      return num_messages_sent;
+    };
+
+    plan_->scheduleProcessors();
+
+    const auto get_time_property_ms = [] (const std::string& property_string) {
+      int64_t value;
+      org::apache::nifi::minifi::core::TimeUnit unit;
+      REQUIRE(org::apache::nifi::minifi::core::Property::StringToTime(property_string, value, unit));
+      int64_t value_as_ms;
+      REQUIRE(org::apache::nifi::minifi::core::Property::ConvertTimeUnitToMS(value, unit, value_as_ms));
+      return value_as_ms;
+    };
+
+    std::thread producer_thread(producer_loop);
+    CHECK_NOTHROW(plan_->runNextProcessor());
+    producer_loop_stop = true;
+    producer_thread.join();
+
+    std::size_t num_flow_files_produced = plan_->getNumFlowFileProducedByCurrentProcessor();
+
+    const uint64_t max_poll_time_ms = get_time_property_ms(max_poll_time.value_or(ConsumeKafka::DEFAULT_MAX_POLL_TIME));
+    const uint64_t max_poll_records_value = max_poll_records ? std::stoi(max_poll_records.value()) : ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+    const uint64_t exp_lower_bound = std::min(max_poll_time_ms / msg_periodicity_ms - 2, max_poll_records_value);
+    const uint64_t exp_upper_bound = std::min(max_poll_time_ms / msg_periodicity_ms + 2, max_poll_records_value);
+    logger_->log_debug("Max poll time: %d, Max poll records: %d, Exp. flowfiles produced: (min: %d, max: %d), actual: %d",
+        max_poll_time_ms, max_poll_records_value, exp_lower_bound, exp_upper_bound, num_flow_files_produced);
+
+    REQUIRE(exp_lower_bound <= num_flow_files_produced);
+    REQUIRE(num_flow_files_produced <= exp_upper_bound);
+  }
+};
+
+const std::string ConsumeKafkaTest::TEST_FILE_NAME_POSTFIX{ "target_kafka_message.txt" };
+const std::string ConsumeKafkaTest::TEST_MESSAGE_KEY{ "consume_kafka_test_key" };
+const std::string ConsumeKafkaTest::PRODUCER_TOPIC{ "ConsumeKafkaTest" };
+const std::string ConsumeKafkaTest::ATTRIBUTE_FOR_CAPTURING_CONTENT{ "flowfile_content" };
+const std::chrono::seconds ConsumeKafkaTest::MAX_CONSUMEKAFKA_POLL_TIME_SECONDS{ 5 };
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "ConsumeKafka parses and uses kafka topics.", "[ConsumeKafka][Kafka][Topic]") {
+  auto run_tests = [&] (const std::vector<std::string>& messages_on_topic, const std::string& topic_names, const optional<std::string>& topic_name_format) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", topic_names, topic_name_format, {}, "test_group_id", {}, {}, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({ "Ulysses",              "James Joyce"         }, "ConsumeKafkaTest",         {});
+  run_tests({ "The Great Gatsby",     "F. Scott Fitzgerald" }, "ConsumeKafkaTest",         ConsumeKafka::TOPIC_FORMAT_NAMES);
+  run_tests({ "War and Peace",        "Lev Tolstoy"         }, "a,b,c,ConsumeKafkaTest,d", ConsumeKafka::TOPIC_FORMAT_NAMES);
+  run_tests({ "Nineteen Eighty Four", "George Orwell"       }, "ConsumeKafkaTest",         ConsumeKafka::TOPIC_FORMAT_PATTERNS);
+  run_tests({ "Hamlet",               "William Shakespeare" }, "Cons[emu]*KafkaTest",      ConsumeKafka::TOPIC_FORMAT_PATTERNS);
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Offsets are reset to the latest when a consumer starts with non-processed messages.", "[ConsumeKafka][Kafka][OffsetReset]") {
+  auto run_tests = [&] (
+      const std::vector<std::string>& messages_on_topic,
+      const std::vector<KafkaTestProducer::PublishEvent>& transaction_events) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, transaction_events, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  KafkaTestProducer producer("localhost:9092", PRODUCER_TOPIC, false);
+  producer.publish_messages_to_topic({"Dummy messages", "that should be ignored", "due to offset reset on ConsumeKafka startup"}, TEST_MESSAGE_KEY, {PUBLISH, PUBLISH, PUBLISH}, {}, {});
+  run_tests({"Brave New World",  "Aldous Huxley"}, NON_TRANSACTIONAL_MESSAGES);
+  producer.publish_messages_to_topic({"Dummy messages", "that should be ignored", "due to offset reset on ConsumeKafka startup"}, TEST_MESSAGE_KEY, {PUBLISH, PUBLISH, PUBLISH}, {}, {});
+  run_tests({"Call of the Wild", "Jack London"}, NON_TRANSACTIONAL_MESSAGES);
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Key attribute is encoded according to the \"Key Attribute Encoding\" property.", "[ConsumeKafka][Kafka][KeyAttributeEncoding]") {
+  auto run_tests = [&] (const std::vector<std::string>& messages_on_topic, const optional<std::string>& key_attribute_encoding) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, key_attribute_encoding, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+
+  run_tests({ "The Odyssey",          "Ὅμηρος"                        }, {});
+  run_tests({ "Lolita",               "Владимир Владимирович Набоков" }, "utf-8");
+  run_tests({ "Crime and Punishment", "Фёдор Михайлович Достоевский"  }, "hex");
+  run_tests({ "Paradise Lost",        "John Milton"                   }, "hEX");
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Transactional behaviour is supported.", "[ConsumeKafka][Kafka][Transaction]") {
+  auto run_tests = [&] (const std::vector<std::string>& messages_on_topic, const std::vector<KafkaTestProducer::PublishEvent>& transaction_events, const optional<bool>& honor_transactions) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, transaction_events, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({  "Pride and Prejudice", "Jane Austen"      }, SINGLE_COMMITTED_TRANSACTION, {});
+  run_tests({                 "Dune", "Frank Herbert"    },    TWO_SEPARATE_TRANSACTIONS, {});
+  run_tests({      "The Black Sheep", "Honore De Balzac" },    NON_COMMITTED_TRANSACTION, {});
+  run_tests({     "Gospel of Thomas"                     },            COMMIT_AND_CANCEL, {});
+  run_tests({ "Operation Dark Heart"                     },            COMMIT_AND_CANCEL, true);
+  run_tests({               "Brexit"                     },            COMMIT_AND_CANCEL, false);
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Headers on consumed Kafka messages are extracted into attributes if requested on ConsumeKafka.", "[ConsumeKafka][Kafka][Batching][Headers]") {
+  auto run_tests = [&] (
+      const std::vector<std::string>& messages_on_topic,
+      const std::vector<std::pair<std::string, std::string>>& expect_header_attributes,
+      const std::vector<std::pair<std::string, std::string>>& message_headers,
+      const optional<std::string>& headers_to_add_as_attributes,
+      const optional<std::string>& duplicate_header_handling) {
+    single_consumer_with_plain_text_test(true, false, expect_header_attributes, messages_on_topic, NON_TRANSACTIONAL_MESSAGES, message_headers, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, headers_to_add_as_attributes, duplicate_header_handling, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({             "Homeland",   "R. A. Salvatore"},                                      {},             {{{"Contains dark elves"}, {"Yes"}}},         {},                    {});
+  run_tests({             "Magician",  "Raymond E. Feist"},               {{{"Rating"}, {"10/10"}}},                        {{{"Rating"}, {"10/10"}}}, {"Rating"},                    {});
+  run_tests({             "Mistborn", "Brandon Sanderson"},               {{{"Metal"}, {"Copper"}}}, {{{"Metal"}, {"Copper"}}, {{"Metal"}, {"Iron"}}},  {"Metal"},            KEEP_FIRST);
+  run_tests({             "Mistborn", "Brandon Sanderson"},                 {{{"Metal"}, {"Iron"}}}, {{{"Metal"}, {"Copper"}}, {{"Metal"}, {"Iron"}}},  {"Metal"},           KEEP_LATEST);
+  run_tests({             "Mistborn", "Brandon Sanderson"},         {{{"Metal"}, {"Copper, Iron"}}}, {{{"Metal"}, {"Copper"}}, {{"Metal"}, {"Iron"}}},  {"Metal"}, COMMA_SEPARATED_MERGE);
+  run_tests({"The Lord of the Rings",  "J. R. R. Tolkien"}, {{{"Parts"}, {"First, second, third"}}},          {{{"Parts"}, {"First, second, third"}}},  {"Parts"},                    {});
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Messages are separated into multiple flowfiles if the message demarcator is present in the message.", "[ConsumeKafka][Kafka][MessageDemarcator]") {
+  auto run_tests = [&] (
+      const std::vector<std::string>& messages_on_topic,
+      const optional<std::string>& message_demarcator) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, message_demarcator, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({"Barbapapa", "Anette Tison and Talus Taylor"}, "a");
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "The maximum poll records allows ConsumeKafka to combine multiple messages into a single flowfile.", "[ConsumeKafka][Kafka][Batching][MaxPollRecords]") {
+  auto run_tests = [&] (
+      const std::vector<std::string>& messages_on_topic,
+      const std::vector<KafkaTestProducer::PublishEvent>& transaction_events,
+      const optional<std::string>& max_poll_records) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, transaction_events, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, {}, {}, max_poll_records, "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({"The Count of Monte Cristo", "Alexandre Dumas"}, NON_TRANSACTIONAL_MESSAGES, "2");
+
+  const std::vector<std::string> content {
+      "Make const member functions thread safe",
+      "Understand special member function generation",
+      "Use std::unique_ptr for exclusive-ownership resource management",
+      "Use std::shared_ptr for shared-ownership resource management",
+      "Use std::weak_ptr for std::shared_ptr-like pointers that can dangle",
+      "Prefer std::make_unique and std::make_shared to direct use of new",
+      "When using the Pimpl Idiom, define special member functions inthe implementation file",
+      "Understand std::move and std::forward",
+      "Distinguish universal references from rvalue references",
+      "Use std::move on rvalue references, std::forward on universal references",
+      "Avoid overloading on universal references",
+      "Familiarize yourself with alternatives to overloading on universal references",
+      "Understand reference collapsing",
+      "Assume that move operations are not present, not cheap, and not used",
+      "Familiarize yourself with perfect forwarding failure cases",
+      "Avoid default capture modes",
+      "Use init capture to move objects into closures",
+      "Use decltype on auto&& parameters to std::forward them",
+      "Prefer lambdas to std::bind",
+      "Prefer task-based programming to thread-based" };
+  const std::vector<KafkaTestProducer::PublishEvent> transaction_events(content.size(), PUBLISH);
+  run_tests(content, transaction_events, "5");
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Non-plain text security context throws scheduling exceptions.", "[ConsumeKafka][Kafka][SecurityProtocol]") {
+  single_consumer_with_plain_text_test(false, false, {}, { "Miyamoto Musashi", "Eiji Yoshikawa" }, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", ConsumeKafka::SECURITY_PROTOCOL_SSL, "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Acceptable values for message header and key attribute encoding are \"UTF-8\" and \"hex\".", "[ConsumeKafka][Kafka][InvalidEncoding]") {
+  single_consumer_with_plain_text_test(false, false, {}, {                           "Shogun", "James Clavell" }, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, "UTF-32", {},       {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  single_consumer_with_plain_text_test(false, false, {}, { "Alice's Adventures in Wonderland", "Lewis Carroll" }, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {},       {}, {}, "UTF-32", {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+}
+
+TEST_CASE_METHOD(ConsumeKafkaContinuousPublishingTest, "ConsumeKafka can spend no more time polling than allowed in the maximum poll time property.", "[ConsumeKafka][Kafka][Batching][MaxPollTime]") {
+  auto run_tests = [&] (
+      const uint64_t msg_periodicity_ms,
+      const optional<std::string>& max_poll_records,
+      const optional<std::string>& max_poll_time,
+      const optional<std::string>& session_timeout) {
+    single_consumer_with_continuous_message_producing(msg_periodicity_ms, "localhost:9092", "test_group_id", max_poll_records, max_poll_time, session_timeout);
+  };
+  // For some reason, a session time-out of a few seconds does not work at all, 10 seconds seems to be stable
+  run_tests(300, "20", "3 seconds", "10000 ms");

Review comment:
       OK, thanks -- in any case, half a second is fine.

##########
File path: extensions/librdkafka/tests/ConsumeKafkaTests.cpp
##########
@@ -0,0 +1,590 @@
+
+/**
+ * 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 "../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(), "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 = "Failed to create Kafka producer" + std::string{ errstr.data() };
+      throw std::runtime_error(error_msg);
+    }
+
+    // The last argument is a config here, but it is already owned by the producer. 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> CANCELLED_TRANSACTION        { 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) {
+    reInitialize();
+
+    // Consumer chain
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+    std::shared_ptr<core::Processor> extract_text  = plan_->addProcessor("ExtractText", "extract_text", {success}, false);
+
+    // Set up connections
+    plan_->addConnection(consume_kafka, success, extract_text);
+    extract_text->setAutoTerminatedRelationships({success});
+
+    const auto bool_to_string = [] (const bool b) -> std::string { return b ? "true" : "false"; };
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::SecurityProtocol.getName(), security_protocol);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), topic_names);
+
+    optional_set_property(consume_kafka, ConsumeKafka::TopicNameFormat.getName(), topic_name_format);
+    optional_set_property(consume_kafka, ConsumeKafka::HonorTransactions.getName(), honor_transactions | utils::map(bool_to_string));
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+    optional_set_property(consume_kafka, ConsumeKafka::OffsetReset.getName(), offset_reset);
+    optional_set_property(consume_kafka, ConsumeKafka::KeyAttributeEncoding.getName(), key_attribute_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageDemarcator.getName(), message_demarcator);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageHeaderEncoding.getName(), message_header_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::HeadersToAddAsAttributes.getName(), headers_to_add_as_attributes);
+    optional_set_property(consume_kafka, ConsumeKafka::DuplicateHeaderHandling.getName(), duplicate_header_handling);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    plan_->setProperty(extract_text, ExtractText::Attribute.getName(), ATTRIBUTE_FOR_CAPTURING_CONTENT);
+
+    if (!expect_config_valid) {
+      REQUIRE_THROWS(plan_->scheduleProcessor(consume_kafka));
+      return;
+    } else {
+      plan_->scheduleProcessors();
+    }
+
+    std::unique_ptr<rd_kafka_conf_t, utils::rd_kafka_conf_deleter> conf_;
+    std::unique_ptr<rd_kafka_t, utils::rd_kafka_consumer_deleter> consumer_;
+
+    const bool is_transactional = std::count(transaction_events.cbegin(), transaction_events.cend(), KafkaTestProducer::PublishEvent::TRANSACTION_START);
+    const bool transactions_committed = transaction_events.back() == KafkaTestProducer::PublishEvent::TRANSACTION_COMMIT;
+
+    KafkaTestProducer producer(kafka_brokers, PRODUCER_TOPIC, is_transactional);
+    producer.publish_messages_to_topic(messages_on_topic, TEST_MESSAGE_KEY, transaction_events, message_headers, message_header_encoding);
+
+
+    std::vector<std::shared_ptr<core::FlowFile>> flow_files_produced;
+    for (std::size_t num_expected_messages_processed = 0; num_expected_messages_processed < messages_on_topic.size(); num_expected_messages_processed += std::stoi(max_poll_records.value_or("1"))) {
+      plan_->increment_location();
+      if ((honor_transactions && false == honor_transactions.value()) || (is_transactional && !transactions_committed)) {
+        INFO("Non-committed messages received.");
+        REQUIRE(false == plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+        return;
+      }
+      {
+        SCOPED_INFO("ConsumeKafka timed out when waiting to receive the message published to the kafka broker.");
+        REQUIRE(plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+      }
+      std::size_t num_flow_files_produced = plan_->getNumFlowFileProducedByCurrentProcessor();
+      plan_->increment_location();
+      for (std::size_t times_extract_text_run = 0; times_extract_text_run < num_flow_files_produced; ++times_extract_text_run) {
+        plan_->runCurrentProcessor();  // ExtractText
+        std::shared_ptr<core::FlowFile> flow_file = plan_->getFlowFileProducedByCurrentProcessor();
+        for (const auto& exp_header : expect_header_attributes) {
+          SCOPED_INFO("ConsumeKafka did not produce the expected flowfile attribute from message header: " << exp_header.first << ".");
+          const auto header_attr_opt = flow_file->getAttribute(exp_header.first);
+          REQUIRE(header_attr_opt);
+          REQUIRE(exp_header.second == header_attr_opt.value().get());
+        }
+        {
+          SCOPED_INFO("Message key is missing or incorrect (potential encoding mismatch).");
+          REQUIRE(TEST_MESSAGE_KEY == decode_key(flow_file->getAttribute(ConsumeKafka::KAFKA_MESSAGE_KEY_ATTR).value().get(), key_attribute_encoding));
+          REQUIRE("1" == flow_file->getAttribute(ConsumeKafka::KAFKA_COUNT_ATTR).value().get());
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_OFFSET_ATTR));
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_PARTITION_ATTR));
+          REQUIRE(PRODUCER_TOPIC == flow_file->getAttribute(ConsumeKafka::KAFKA_TOPIC_ATTR).value().get());
+        }
+        flow_files_produced.emplace_back(std::move(flow_file));
+      }
+      plan_->reset_location();
+    }
+
+    const auto contentOrderOfFlowFile = [&] (const std::shared_ptr<core::FlowFile>& lhs, const std::shared_ptr<core::FlowFile>& rhs) {
+      return lhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() < rhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get();
+    };
+    {
+      SCOPED_INFO("The flowfiles generated by ConsumeKafka are invalid (probably nullptr).");
+      REQUIRE_NOTHROW(std::sort(flow_files_produced.begin(), flow_files_produced.end(), contentOrderOfFlowFile));
+    }
+    std::vector<std::string> sorted_split_messages = sort_and_split_messages(messages_on_topic, message_demarcator);
+    const auto flow_file_content_matches_message = [&] (const std::shared_ptr<core::FlowFile>& flowfile, const std::string message) {
+      return flowfile->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() == message;
+    };
+
+    logger_->log_debug("************");
+    std::string expected = "Expected: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      expected += sorted_split_messages[i] + ", ";
+    }
+    std::string   actual = "  Actual: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      actual += flow_files_produced[i]->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() + ", ";
+    }
+    logger_->log_debug("%s", expected.c_str());
+    logger_->log_debug("%s", actual.c_str());
+    logger_->log_debug("************");
+
+    INFO("The messages received by ConsumeKafka do not match those published");
+    REQUIRE(std::equal(flow_files_produced.begin(), flow_files_produced.end(), sorted_split_messages.begin(), flow_file_content_matches_message));
+  }
+};
+
+class ConsumeKafkaContinuousPublishingTest : public ConsumeKafkaTest {
+ public:
+  ConsumeKafkaContinuousPublishingTest() : ConsumeKafkaTest() {}
+  virtual ~ConsumeKafkaContinuousPublishingTest() {
+    logTestController_.reset();
+  }
+
+  void single_consumer_with_continuous_message_producing(
+      const uint64_t msg_periodicity_ms,
+      const std::string& kafka_brokers,
+      const optional<std::string>& group_id,
+      const optional<std::string>& max_poll_records,
+      const optional<std::string>& max_poll_time,
+      const optional<std::string>& session_timeout) {
+    reInitialize();
+
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+
+    plan_->setProperty(consume_kafka, "allow.auto.create.topics", "true", true);  // Seems like the topic tests work without this
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), PRODUCER_TOPIC);
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    consume_kafka->setAutoTerminatedRelationships({success});
+
+    KafkaTestProducer producer("localhost:9092", PRODUCER_TOPIC, /* transactional = */ false);
+
+    std::atomic_bool producer_loop_stop{ false };
+    const auto producer_loop = [&] {
+      std::size_t num_messages_sent = 0;  // When on C++14 make this an initialized lambda capture

Review comment:
       this comment can be removed

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,582 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger->should_log(core::logging::LOG_LEVEL::info)) {
+        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
+      if (logger->should_log(core::logging::LOG_LEVEL::info)) {

Review comment:
       Why `info`?  `print_topics_list` logs the topics on Debug level.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] fgerlits commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
fgerlits commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r552072783



##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);
+  }
+}
+
+void print_topics_list(std::shared_ptr<logging::Logger> logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger->log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld]",

Review comment:
       is the `]` at the end of the line there on purpose?

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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());

Review comment:
       Not all of these properties are marked as Required above (and in PROCESSORS.md), eg. TopicNames and GroupID aren't.

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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() };

Review comment:
       This will create a string of length 512 with embedded nulls.  If we are sure Kafka returns a null-terminated error message, we can do `const std::string error_msg{errstr.data()};`

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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*/) {

Review comment:
       this function could be in an anonymous namespace




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r577488062



##########
File path: libminifi/test/TestBase.cpp
##########
@@ -61,44 +62,41 @@ TestPlan::~TestPlan() {
   for (auto& processor : configured_processors_) {
     processor->setScheduledState(core::ScheduledState::STOPPED);
   }
+  for (auto& connection : relationships_) {
+    // This is a patch solving circular references between processors and connections
+    connection->setSource(nullptr);
+    connection->setDestination(nullptr);
+  }
   controller_services_provider_->clearControllerServices();
 }
 
 std::shared_ptr<core::Processor> TestPlan::addProcessor(const std::shared_ptr<core::Processor> &processor, const std::string& /*name*/, const std::initializer_list<core::Relationship>& relationships,
-                                                        bool linkToPrevious) {
+    bool linkToPrevious) {
   if (finalized) {
     return nullptr;
   }
   std::lock_guard<std::recursive_mutex> guard(mutex);
-
   utils::Identifier uuid = utils::IdGenerator::getIdGenerator()->generate();
-
   processor->setStreamFactory(stream_factory);
   // initialize the processor
   processor->initialize();
   processor->setFlowIdentifier(flow_version_->getFlowIdentifier());
-
   processor_mapping_[processor->getUUID()] = processor;
-
   if (!linkToPrevious) {
     termination_ = *(relationships.begin());
   } else {
     std::shared_ptr<core::Processor> last = processor_queue_.back();
-
     if (last == nullptr) {
       last = processor;
       termination_ = *(relationships.begin());
     }
-
     std::stringstream connection_name;
     connection_name << last->getUUIDStr() << "-to-" << processor->getUUIDStr();
-    logger_->log_info("Creating %s connection for proc %d", connection_name.str(), processor_queue_.size() + 1);

Review comment:
       Restored the original. I am fine with having this log, but seems more of an annoyance than an useful log line. Other parts creating `minifi::Connection` objects don't log so seeing log lines on part of the connections being created could be misleading.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r567637259



##########
File path: libminifi/test/TestBase.cpp
##########
@@ -62,78 +63,51 @@ TestPlan::~TestPlan() {
   for (auto& processor : configured_processors_) {
     processor->setScheduledState(core::ScheduledState::STOPPED);
   }
+  for (auto& connection : relationships_) {
+    // This is a patch solving circular references between processors and connections
+    connection->setSource(nullptr);
+    connection->setDestination(nullptr);
+  }
   controller_services_provider_->clearControllerServices();
 }
 
 std::shared_ptr<core::Processor> TestPlan::addProcessor(const std::shared_ptr<core::Processor> &processor, const std::string &name, const std::initializer_list<core::Relationship>& relationships,
-                                                        bool linkToPrevious) {
+    bool linkToPrevious) {
   if (finalized) {
     return nullptr;
   }
   std::lock_guard<std::recursive_mutex> guard(mutex);
-
   utils::Identifier uuid = utils::IdGenerator::getIdGenerator()->generate();
-
   processor->setStreamFactory(stream_factory);
   // initialize the processor
   processor->initialize();
   processor->setFlowIdentifier(flow_version_->getFlowIdentifier());
-
   processor_mapping_[processor->getUUID()] = processor;
-
   if (!linkToPrevious) {
     termination_ = *(relationships.begin());
   } else {
     std::shared_ptr<core::Processor> last = processor_queue_.back();
-
     if (last == nullptr) {
       last = processor;
       termination_ = *(relationships.begin());
     }
-
-    std::stringstream connection_name;
-    connection_name << last->getUUIDStr() << "-to-" << processor->getUUIDStr();
-    logger_->log_info("Creating %s connection for proc %d", connection_name.str(), processor_queue_.size() + 1);
-    std::shared_ptr<minifi::Connection> connection = std::make_shared<minifi::Connection>(flow_repo_, content_repo_, connection_name.str());
-
     for (const auto& relationship : relationships) {
-      connection->addRelationship(relationship);
-    }
-
-    // link the connections so that we can test results at the end for this
-    connection->setSource(last);
-    connection->setDestination(processor);
-
-    connection->setSourceUUID(last->getUUID());
-    connection->setDestinationUUID(processor->getUUID());
-    last->addConnection(connection);
-    if (last != processor) {
-      processor->addConnection(connection);
+      addConnection(last, relationship, processor);

Review comment:
       I think this is the same behavior, just extracted to a separate function.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553392644



##########
File path: libminifi/test/TestBase.h
##########
@@ -200,7 +199,7 @@ class LogTestController {
     }
     my_properties_->set("logger.root", "ERROR,ostream");
     my_properties_->set("logger." + core::getClassName<LogTestController>(), "INFO");
-    my_properties_->set("logger." + core::getClassName<logging::LoggerConfiguration>(), "DEBUG");
+    my_properties_->set("logger." + core::getClassName<logging::LoggerConfiguration>(), "INFO");

Review comment:
       It is no longer an issue.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r560819443



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,569 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));
+  }
+  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::steady_clock::now();
+  auto elapsed = std::chrono::steady_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());

Review comment:
       Sorry, I assumed I was at least consistent with my habit of putting spaces around arrows. Search and replace did not work on occurences where the space was on one side only.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553793221



##########
File path: libminifi/include/utils/ProcessorConfigUtils.h
##########
@@ -0,0 +1,80 @@
+/**
+ * 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) {
+  std::string value;
+  if (!context->getProperty(property_name, value)) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property missing or invalid");
+  }
+  return value;
+}
+
+std::vector<std::string> listFromCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  std::string property_string;
+  context->getProperty(property_name, property_string);
+  return utils::StringUtils::splitAndTrim(property_string, ",");
+}
+
+std::vector<std::string> listFromRequiredCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  return utils::StringUtils::splitAndTrim(getRequiredPropertyOrThrow(context, property_name), ",");
+}
+
+bool parseBooleanPropertyOrThrow(core::ProcessContext* context, const std::string& property_name) {
+  bool value;
+  std::string value_str = getRequiredPropertyOrThrow(context, property_name);
+  utils::optional<bool> maybe_value = utils::StringUtils::toBool(value_str);
+  if (!maybe_value) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property is invalid: value is " + value_str);

Review comment:
       Does it really matter though? I would be even happy to throw `std::runtime_error` here.

##########
File path: libminifi/include/utils/ProcessorConfigUtils.h
##########
@@ -0,0 +1,80 @@
+/**
+ * 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) {
+  std::string value;
+  if (!context->getProperty(property_name, value)) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property missing or invalid");
+  }
+  return value;
+}
+
+std::vector<std::string> listFromCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  std::string property_string;
+  context->getProperty(property_name, property_string);
+  return utils::StringUtils::splitAndTrim(property_string, ",");
+}
+
+std::vector<std::string> listFromRequiredCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  return utils::StringUtils::splitAndTrim(getRequiredPropertyOrThrow(context, property_name), ",");
+}
+
+bool parseBooleanPropertyOrThrow(core::ProcessContext* context, const std::string& property_name) {
+  bool value;
+  std::string value_str = getRequiredPropertyOrThrow(context, property_name);
+  utils::optional<bool> maybe_value = utils::StringUtils::toBool(value_str);
+  if (!maybe_value) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property is invalid: value is " + value_str);
+  }
+  return maybe_value.value();
+}
+
+std::chrono::milliseconds parseTimePropertyMSOrThrow(core::ProcessContext* context, const std::string& property_name) {
+  core::TimeUnit unit;
+  uint64_t time_value_ms;
+  if (!core::Property::StringToTime(getRequiredPropertyOrThrow(context, property_name), time_value_ms, unit) || !core::Property::ConvertTimeUnitToMS(time_value_ms, unit, time_value_ms)) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property missing or invalid");

Review comment:
       Updated the error message.

##########
File path: libminifi/include/utils/ProcessorConfigUtils.h
##########
@@ -0,0 +1,80 @@
+/**
+ * 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) {
+  std::string value;
+  if (!context->getProperty(property_name, value)) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property missing or invalid");
+  }
+  return value;
+}
+
+std::vector<std::string> listFromCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  std::string property_string;
+  context->getProperty(property_name, property_string);
+  return utils::StringUtils::splitAndTrim(property_string, ",");
+}
+
+std::vector<std::string> listFromRequiredCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  return utils::StringUtils::splitAndTrim(getRequiredPropertyOrThrow(context, property_name), ",");
+}
+
+bool parseBooleanPropertyOrThrow(core::ProcessContext* context, const std::string& property_name) {
+  bool value;
+  std::string value_str = getRequiredPropertyOrThrow(context, property_name);
+  utils::optional<bool> maybe_value = utils::StringUtils::toBool(value_str);
+  if (!maybe_value) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property is invalid: value is " + value_str);
+  }
+  return maybe_value.value();
+}
+
+std::chrono::milliseconds parseTimePropertyMSOrThrow(core::ProcessContext* context, const std::string& property_name) {
+  core::TimeUnit unit;
+  uint64_t time_value_ms;
+  if (!core::Property::StringToTime(getRequiredPropertyOrThrow(context, property_name), time_value_ms, unit) || !core::Property::ConvertTimeUnitToMS(time_value_ms, unit, time_value_ms)) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property missing or invalid");
+  }
+  return std::chrono::milliseconds(time_value_ms);
+}
+
+utils::optional<uint64_t> getOptionalUintProperty(core::ProcessContext* context, const std::string& property_name) {

Review comment:
       Good comment! Updated.

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  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() const {
+  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() const {
+  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) const {
+  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) const {
+  // 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()) {

Review comment:
       Added a 200 character limit on the logs.

##########
File path: libminifi/include/core/TypedValues.h
##########
@@ -106,6 +106,8 @@ class TimePeriodValue : public TransformableValue, public state::response::UInt6
   static bool StringToTime(std::string input, uint64_t &output, TimeUnit &timeunit) {
     return utils::internal::StringToTime(input, output, timeunit);
   }
+
+  TimePeriodValue& operator=(const TimePeriodValue& other) = default;

Review comment:
       I don't remember anymore :) Removed.

##########
File path: libminifi/include/utils/ProcessorConfigUtils.h
##########
@@ -0,0 +1,80 @@
+/**
+ * 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) {
+  std::string value;
+  if (!context->getProperty(property_name, value)) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property missing or invalid");
+  }
+  return value;
+}
+
+std::vector<std::string> listFromCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  std::string property_string;
+  context->getProperty(property_name, property_string);
+  return utils::StringUtils::splitAndTrim(property_string, ",");
+}
+
+std::vector<std::string> listFromRequiredCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  return utils::StringUtils::splitAndTrim(getRequiredPropertyOrThrow(context, property_name), ",");
+}
+
+bool parseBooleanPropertyOrThrow(core::ProcessContext* context, const std::string& property_name) {
+  bool value;
+  std::string value_str = getRequiredPropertyOrThrow(context, property_name);
+  utils::optional<bool> maybe_value = utils::StringUtils::toBool(value_str);
+  if (!maybe_value) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property is invalid: value is " + value_str);

Review comment:
       Updated to `runtime_error`.

##########
File path: libminifi/test/TestBase.cpp
##########
@@ -62,78 +63,51 @@ TestPlan::~TestPlan() {
   for (auto& processor : configured_processors_) {
     processor->setScheduledState(core::ScheduledState::STOPPED);
   }
+  for (auto& connection : relationships_) {
+    // This is a patch solving circular references between processors and connections
+    connection->setSource(nullptr);
+    connection->setDestination(nullptr);
+  }
   controller_services_provider_->clearControllerServices();
 }
 
 std::shared_ptr<core::Processor> TestPlan::addProcessor(const std::shared_ptr<core::Processor> &processor, const std::string &name, const std::initializer_list<core::Relationship>& relationships,
-                                                        bool linkToPrevious) {
+    bool linkToPrevious) {
   if (finalized) {
     return nullptr;
   }
   std::lock_guard<std::recursive_mutex> guard(mutex);
-
   utils::Identifier uuid = utils::IdGenerator::getIdGenerator()->generate();
-
   processor->setStreamFactory(stream_factory);
   // initialize the processor
   processor->initialize();
   processor->setFlowIdentifier(flow_version_->getFlowIdentifier());
-
   processor_mapping_[processor->getUUID()] = processor;
-
   if (!linkToPrevious) {
     termination_ = *(relationships.begin());
   } else {
     std::shared_ptr<core::Processor> last = processor_queue_.back();
-
     if (last == nullptr) {
       last = processor;
       termination_ = *(relationships.begin());
     }
-
-    std::stringstream connection_name;
-    connection_name << last->getUUIDStr() << "-to-" << processor->getUUIDStr();
-    logger_->log_info("Creating %s connection for proc %d", connection_name.str(), processor_queue_.size() + 1);
-    std::shared_ptr<minifi::Connection> connection = std::make_shared<minifi::Connection>(flow_repo_, content_repo_, connection_name.str());
-
     for (const auto& relationship : relationships) {
-      connection->addRelationship(relationship);
-    }
-
-    // link the connections so that we can test results at the end for this
-    connection->setSource(last);
-    connection->setDestination(processor);
-
-    connection->setSourceUUID(last->getUUID());
-    connection->setDestinationUUID(processor->getUUID());
-    last->addConnection(connection);
-    if (last != processor) {
-      processor->addConnection(connection);
+      addConnection(last, relationship, processor);

Review comment:
       I am not sure I understand the question, but It is a prerequisite if you want to connect up multiple relationships separately.

##########
File path: libminifi/test/TestBase.cpp
##########
@@ -62,78 +63,51 @@ TestPlan::~TestPlan() {
   for (auto& processor : configured_processors_) {
     processor->setScheduledState(core::ScheduledState::STOPPED);
   }
+  for (auto& connection : relationships_) {
+    // This is a patch solving circular references between processors and connections
+    connection->setSource(nullptr);
+    connection->setDestination(nullptr);
+  }
   controller_services_provider_->clearControllerServices();
 }
 
 std::shared_ptr<core::Processor> TestPlan::addProcessor(const std::shared_ptr<core::Processor> &processor, const std::string &name, const std::initializer_list<core::Relationship>& relationships,
-                                                        bool linkToPrevious) {
+    bool linkToPrevious) {
   if (finalized) {
     return nullptr;
   }
   std::lock_guard<std::recursive_mutex> guard(mutex);
-
   utils::Identifier uuid = utils::IdGenerator::getIdGenerator()->generate();
-
   processor->setStreamFactory(stream_factory);
   // initialize the processor
   processor->initialize();
   processor->setFlowIdentifier(flow_version_->getFlowIdentifier());
-
   processor_mapping_[processor->getUUID()] = processor;
-
   if (!linkToPrevious) {
     termination_ = *(relationships.begin());
   } else {
     std::shared_ptr<core::Processor> last = processor_queue_.back();
-
     if (last == nullptr) {
       last = processor;
       termination_ = *(relationships.begin());
     }
-
-    std::stringstream connection_name;
-    connection_name << last->getUUIDStr() << "-to-" << processor->getUUIDStr();
-    logger_->log_info("Creating %s connection for proc %d", connection_name.str(), processor_queue_.size() + 1);
-    std::shared_ptr<minifi::Connection> connection = std::make_shared<minifi::Connection>(flow_repo_, content_repo_, connection_name.str());
-
     for (const auto& relationship : relationships) {
-      connection->addRelationship(relationship);
-    }
-
-    // link the connections so that we can test results at the end for this
-    connection->setSource(last);
-    connection->setDestination(processor);
-
-    connection->setSourceUUID(last->getUUID());
-    connection->setDestinationUUID(processor->getUUID());
-    last->addConnection(connection);
-    if (last != processor) {
-      processor->addConnection(connection);
+      addConnection(last, relationship, processor);

Review comment:
       I am not sure I understand the question, but is it not a prerequisite if you want to connect up multiple relationships separately?

##########
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;
+    // }
+    rd_kafka_destroy(ptr);
+  }
+};
+
+struct rd_kafka_consumer_deleter {
+  void operator()(rd_kafka_t* ptr) const noexcept {
+    rd_kafka_consumer_close(ptr);
+    rd_kafka_destroy(ptr);
+  }
+};
+
+struct rd_kafka_topic_partition_list_deleter {
+  void operator()(rd_kafka_topic_partition_list_t* ptr) const noexcept { rd_kafka_topic_partition_list_destroy(ptr); }
+};
+
+struct rd_kafka_topic_conf_deleter {
+  void operator()(rd_kafka_topic_conf_t* ptr) const noexcept { rd_kafka_topic_conf_destroy(ptr); }
+};
+struct rd_kafka_topic_deleter {
+  void operator()(rd_kafka_topic_t* ptr) const noexcept { rd_kafka_topic_destroy(ptr); }
+};
+
+struct rd_kafka_message_deleter {
+  void operator()(rd_kafka_message_t* ptr) const noexcept { rd_kafka_message_destroy(ptr); }
+};
+
+struct rd_kafka_headers_deleter {
+  void operator()(rd_kafka_headers_t* ptr) const noexcept { rd_kafka_headers_destroy(ptr); }
+};
+
+template <typename T>
+void kafka_headers_for_each(const rd_kafka_headers_t* headers, T&& key_value_handle) {
+  const char *key;  // Null terminated, not to be freed
+  const void *value;
+  std::size_t size;
+  for (std::size_t i = 0; RD_KAFKA_RESP_ERR_NO_ERROR == rd_kafka_header_get_all(headers, i, &key, &value, &size); ++i) {
+    std::forward<T>(key_value_handle)(std::string(key), std::string(static_cast<const char*>(value), size));

Review comment:
       Good call! Updated.

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);
+  }
+}
+
+void print_topics_list(std::shared_ptr<logging::Logger> logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger->log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld]",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, const std::shared_ptr<logging::Logger>& logger) {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    const std::string error_msg = "ConsumeKafka: received error message from broker. Librdkafka error msg: " + std::string(rd_kafka_err2str(rkmessage->err));
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, error_msg);
+  }
+  std::string topicName = rd_kafka_topic_name(rkmessage->rkt);
+  std::string message(reinterpret_cast<char*>(rkmessage->payload), rkmessage->len);
+  const char* key = reinterpret_cast<const char*>(rkmessage->key);
+  const std::size_t key_len = rkmessage->key_len;
+  rd_kafka_timestamp_type_t tstype;
+  int64_t timestamp;
+  timestamp = rd_kafka_message_timestamp(rkmessage, &tstype);
+  const char *tsname = "?";
+  if (tstype != RD_KAFKA_TIMESTAMP_NOT_AVAILABLE) {
+    if (tstype == RD_KAFKA_TIMESTAMP_CREATE_TIME) {
+      tsname = "create time";
+    } else if (tstype == RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME) {
+      tsname = "log append time";
+    }
+  }
+  const int64_t seconds_since_timestamp = timestamp ? static_cast<int64_t>(time(NULL)) - static_cast<int64_t>(timestamp / 1000) : 0;
+
+  std::string headers_as_string;
+  rd_kafka_headers_t* hdrs;
+  const rd_kafka_resp_err_t get_header_response = rd_kafka_message_headers(rkmessage, &hdrs);
+  if (RD_KAFKA_RESP_ERR_NO_ERROR == get_header_response) {
+    std::vector<std::string> header_list;
+    kafka_headers_for_each(hdrs, [&] (const std::string& key, const std::string& val) { header_list.emplace_back(key + ": " + val); });
+    headers_as_string = StringUtils::join(", ", header_list);
+  } else if (RD_KAFKA_RESP_ERR__NOENT != 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::string message_as_string;
+  message_as_string += "[Topic](" + topicName + "), ";
+  message_as_string += "[Key](" + (key != nullptr ? std::string(key, key_len) : std::string("[None]")) + "), ";
+  message_as_string += "[Offset](" +  std::to_string(rkmessage->offset) + "), ";
+  message_as_string += "[Message Length](" + std::to_string(rkmessage->len) + "), ";
+  message_as_string += "[Timestamp](" + std::string(tsname) + " " + std::to_string(timestamp) + " (" + std::to_string(seconds_since_timestamp) + " s ago)), ";
+  message_as_string += "[Headers](";
+  message_as_string += headers_as_string + "\n";
+  message_as_string += "[Payload](" + message + ")";
+
+  logger -> log_debug("Message: %s", message_as_string.c_str());

Review comment:
       Seems like this was an outdated commit, fixed since then.

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);

Review comment:
       Added missing " :".

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);
+  }
+}
+
+void print_topics_list(std::shared_ptr<logging::Logger> logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger->log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld]",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, const std::shared_ptr<logging::Logger>& logger) {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    const std::string error_msg = "ConsumeKafka: received error message from broker. Librdkafka error msg: " + std::string(rd_kafka_err2str(rkmessage->err));
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, error_msg);
+  }
+  std::string topicName = rd_kafka_topic_name(rkmessage->rkt);
+  std::string message(reinterpret_cast<char*>(rkmessage->payload), rkmessage->len);
+  const char* key = reinterpret_cast<const char*>(rkmessage->key);
+  const std::size_t key_len = rkmessage->key_len;
+  rd_kafka_timestamp_type_t tstype;
+  int64_t timestamp;
+  timestamp = rd_kafka_message_timestamp(rkmessage, &tstype);
+  const char *tsname = "?";
+  if (tstype != RD_KAFKA_TIMESTAMP_NOT_AVAILABLE) {
+    if (tstype == RD_KAFKA_TIMESTAMP_CREATE_TIME) {
+      tsname = "create time";
+    } else if (tstype == RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME) {
+      tsname = "log append time";
+    }
+  }
+  const int64_t seconds_since_timestamp = timestamp ? static_cast<int64_t>(time(NULL)) - static_cast<int64_t>(timestamp / 1000) : 0;
+
+  std::string headers_as_string;
+  rd_kafka_headers_t* hdrs;
+  const rd_kafka_resp_err_t get_header_response = rd_kafka_message_headers(rkmessage, &hdrs);
+  if (RD_KAFKA_RESP_ERR_NO_ERROR == get_header_response) {
+    std::vector<std::string> header_list;
+    kafka_headers_for_each(hdrs, [&] (const std::string& key, const std::string& val) { header_list.emplace_back(key + ": " + val); });
+    headers_as_string = StringUtils::join(", ", header_list);
+  } else if (RD_KAFKA_RESP_ERR__NOENT != 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::string message_as_string;
+  message_as_string += "[Topic](" + topicName + "), ";
+  message_as_string += "[Key](" + (key != nullptr ? std::string(key, key_len) : std::string("[None]")) + "), ";
+  message_as_string += "[Offset](" +  std::to_string(rkmessage->offset) + "), ";
+  message_as_string += "[Message Length](" + std::to_string(rkmessage->len) + "), ";
+  message_as_string += "[Timestamp](" + std::string(tsname) + " " + std::to_string(timestamp) + " (" + std::to_string(seconds_since_timestamp) + " s ago)), ";
+  message_as_string += "[Headers](";
+  message_as_string += headers_as_string + "\n";
+  message_as_string += "[Payload](" + message + ")";
+
+  logger -> log_debug("Message: %s", message_as_string.c_str());
+}
+
+std::string get_encoded_string(const std::string& input, KafkaEncoding encoding) {
+  switch (encoding) {
+    case KafkaEncoding::UTF8:
+      return input;
+    case KafkaEncoding::HEX:
+      return StringUtils::to_hex(input, /* uppercase = */ true);
+  }
+  throw std::runtime_error("Invalid encoding selected for encoding.");

Review comment:
       Updated but with simple string concatenation, I usually refrain from using fmt unless there is a practical need.

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,569 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));

Review comment:
       I don't mind concatenation when building an error.

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,569 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));

Review comment:
       I don't mind concatenation when building an error. Introducing a framework dependence here seems overkill, and `join_pack` seems unneccessarily complex. Can I leave it like this?

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);
+  }
+}
+
+void print_topics_list(std::shared_ptr<logging::Logger> logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger->log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld]",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, const std::shared_ptr<logging::Logger>& logger) {

Review comment:
       Can a const ref on a shared pointer even manipulate lifetime or ownership? I don't mind changing so updated them as requested.

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,569 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));
+  }
+  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::steady_clock::now();
+  auto elapsed = std::chrono::steady_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());
+    std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>
+      message { rd_kafka_consumer_poll(consumer_.get(), std::chrono::duration_cast<std::chrono::milliseconds>(max_poll_time_milliseconds_ - elapsed).count()), utils::rd_kafka_message_deleter() };
+    if (!message) {
+      break;
+    }
+    if (RD_KAFKA_RESP_ERR_NO_ERROR != message->err) {
+      logger_->log_error("Received message with error %d: %s", message->err, rd_kafka_err2str(message->err));
+      break;
+    }
+    utils::print_kafka_message(message.get(), logger_);
+    messages.emplace_back(std::move(message));
+    elapsed = std::chrono::steady_clock::now() - start;
+  }
+  return messages;
+}
+
+utils::KafkaEncoding ConsumeKafka::key_attr_encoding_attr_to_enum() const {
+  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() const {
+  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, "\"Message Header Encoding\" property not recognized.");
+}
+
+std::string ConsumeKafka::resolve_duplicate_headers(const std::vector<std::string>& matching_headers) const {
+  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) const {
+  // 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) const {
+  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_) {
+      ret = stream->write(data_,  gsl::narrow<int>(dataSize_));
+    }
+    return ret;
+  }
+};
+
+void ConsumeKafka::add_kafka_attributes_to_flowfile(std::shared_ptr<FlowFileRecord>& flow_file, const rd_kafka_message_t* message) const {
+  // 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, 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));
+}
+
+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) const {
+  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("Message received contains no data.");
+      continue;
+    }

Review comment:
       Deleted these lines.

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,569 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));
+  }
+  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::steady_clock::now();
+  auto elapsed = std::chrono::steady_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());
+    std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>
+      message { rd_kafka_consumer_poll(consumer_.get(), std::chrono::duration_cast<std::chrono::milliseconds>(max_poll_time_milliseconds_ - elapsed).count()), utils::rd_kafka_message_deleter() };
+    if (!message) {
+      break;
+    }
+    if (RD_KAFKA_RESP_ERR_NO_ERROR != message->err) {
+      logger_->log_error("Received message with error %d: %s", message->err, rd_kafka_err2str(message->err));
+      break;
+    }
+    utils::print_kafka_message(message.get(), logger_);
+    messages.emplace_back(std::move(message));
+    elapsed = std::chrono::steady_clock::now() - start;
+  }
+  return messages;
+}
+
+utils::KafkaEncoding ConsumeKafka::key_attr_encoding_attr_to_enum() const {
+  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() const {
+  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, "\"Message Header Encoding\" property not recognized.");
+}
+
+std::string ConsumeKafka::resolve_duplicate_headers(const std::vector<std::string>& matching_headers) const {
+  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) const {
+  // 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) const {
+  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_) {
+      ret = stream->write(data_,  gsl::narrow<int>(dataSize_));
+    }
+    return ret;
+  }
+};
+
+void ConsumeKafka::add_kafka_attributes_to_flowfile(std::shared_ptr<FlowFileRecord>& flow_file, const rd_kafka_message_t* message) const {
+  // 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, 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));
+}
+
+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) const {
+  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("Message received contains no data.");
+      continue;
+    }
+
+    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 { message_content };
+    if (message_demarcator_.size()) {
+      split_message = utils::StringUtils::split(message_content, message_demarcator_);
+    }

Review comment:
       Went with a ternary instead:
   ```c++
   std::vector<std::string> split_message{ message_demarcator_.size() ?
     utils::StringUtils::split(message_content, message_demarcator_) :
     std::vector<std::string>{ message_content }};
   ```

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,569 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));
+  }
+  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::steady_clock::now();
+  auto elapsed = std::chrono::steady_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());
+    std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>
+      message { rd_kafka_consumer_poll(consumer_.get(), std::chrono::duration_cast<std::chrono::milliseconds>(max_poll_time_milliseconds_ - elapsed).count()), utils::rd_kafka_message_deleter() };
+    if (!message) {
+      break;
+    }
+    if (RD_KAFKA_RESP_ERR_NO_ERROR != message->err) {
+      logger_->log_error("Received message with error %d: %s", message->err, rd_kafka_err2str(message->err));
+      break;
+    }
+    utils::print_kafka_message(message.get(), logger_);
+    messages.emplace_back(std::move(message));
+    elapsed = std::chrono::steady_clock::now() - start;
+  }
+  return messages;
+}
+
+utils::KafkaEncoding ConsumeKafka::key_attr_encoding_attr_to_enum() const {
+  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() const {
+  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, "\"Message Header Encoding\" property not recognized.");
+}
+
+std::string ConsumeKafka::resolve_duplicate_headers(const std::vector<std::string>& matching_headers) const {
+  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) const {
+  // 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) const {
+  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_) {
+      ret = stream->write(data_,  gsl::narrow<int>(dataSize_));
+    }
+    return ret;
+  }
+};
+
+void ConsumeKafka::add_kafka_attributes_to_flowfile(std::shared_ptr<FlowFileRecord>& flow_file, const rd_kafka_message_t* message) const {
+  // 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, 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));
+}
+
+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) const {
+  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("Message received contains no data.");
+      continue;
+    }
+
+    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 { message_content };
+    if (message_demarcator_.size()) {
+      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) {
+        logger_->log_error("Failed to create flowfile.");
+        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);
+      }
+      add_kafka_attributes_to_flowfile(flow_file, message.get());
+      flow_files_created.emplace_back(std::move(flow_file));
+    }
+  }
+  return flow_files_created;
+}
+
+void ConsumeKafka::onTrigger(core::ProcessContext* /* context */, core::ProcessSession* session) {
+  logger_->log_debug("ConsumeKafka onTrigger");
+
+  std::vector<std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>> messages = poll_kafka_messages();
+
+  std::vector<std::shared_ptr<FlowFileRecord>> flow_files_created = transform_messages_into_flowfiles(messages, session);
+  if (flow_files_created.empty()) {
+    return;
+  }
+  for (const auto& flow_file : flow_files_created) {
+    session->transfer(flow_file, Success);
+  }
+  session->commit();
+  // Commit the offset from the latest message only
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rd_kafka_commit_message(consumer_.get(), messages.back().get(), /* async = */ 0)) {
+    logger_ -> log_error("Committing offsets failed.");

Review comment:
       Seems like this comment is outdated.

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);
+  }
+}
+
+void print_topics_list(std::shared_ptr<logging::Logger> logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger->log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld]",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, const std::shared_ptr<logging::Logger>& logger) {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    const std::string error_msg = "ConsumeKafka: received error message from broker. Librdkafka error msg: " + std::string(rd_kafka_err2str(rkmessage->err));
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, error_msg);
+  }
+  std::string topicName = rd_kafka_topic_name(rkmessage->rkt);
+  std::string message(reinterpret_cast<char*>(rkmessage->payload), rkmessage->len);
+  const char* key = reinterpret_cast<const char*>(rkmessage->key);
+  const std::size_t key_len = rkmessage->key_len;
+  rd_kafka_timestamp_type_t tstype;
+  int64_t timestamp;
+  timestamp = rd_kafka_message_timestamp(rkmessage, &tstype);
+  const char *tsname = "?";
+  if (tstype != RD_KAFKA_TIMESTAMP_NOT_AVAILABLE) {
+    if (tstype == RD_KAFKA_TIMESTAMP_CREATE_TIME) {
+      tsname = "create time";
+    } else if (tstype == RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME) {
+      tsname = "log append time";
+    }
+  }
+  const int64_t seconds_since_timestamp = timestamp ? static_cast<int64_t>(time(NULL)) - static_cast<int64_t>(timestamp / 1000) : 0;

Review comment:
       It does print the absolute one as well.

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);
+  }
+}
+
+void print_topics_list(std::shared_ptr<logging::Logger> logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger->log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld]",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, const std::shared_ptr<logging::Logger>& logger) {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    const std::string error_msg = "ConsumeKafka: received error message from broker. Librdkafka error msg: " + std::string(rd_kafka_err2str(rkmessage->err));
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, error_msg);
+  }
+  std::string topicName = rd_kafka_topic_name(rkmessage->rkt);
+  std::string message(reinterpret_cast<char*>(rkmessage->payload), rkmessage->len);
+  const char* key = reinterpret_cast<const char*>(rkmessage->key);
+  const std::size_t key_len = rkmessage->key_len;
+  rd_kafka_timestamp_type_t tstype;
+  int64_t timestamp;
+  timestamp = rd_kafka_message_timestamp(rkmessage, &tstype);
+  const char *tsname = "?";
+  if (tstype != RD_KAFKA_TIMESTAMP_NOT_AVAILABLE) {
+    if (tstype == RD_KAFKA_TIMESTAMP_CREATE_TIME) {
+      tsname = "create time";
+    } else if (tstype == RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME) {
+      tsname = "log append time";
+    }
+  }
+  const int64_t seconds_since_timestamp = timestamp ? static_cast<int64_t>(time(NULL)) - static_cast<int64_t>(timestamp / 1000) : 0;

Review comment:
       It does print the absolute one as well in this format:
   > [Timestamp](create time 1610107476940 (0 s ago)),

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,582 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger->should_log(core::logging::LOG_LEVEL::info)) {
+        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
+      if (logger->should_log(core::logging::LOG_LEVEL::info)) {

Review comment:
       No reason, I made a mistake. Corrected now.

##########
File path: extensions/librdkafka/docker_tests/CMakeLists.txt
##########
@@ -0,0 +1,36 @@
+#

Review comment:
       I mean these will sit in docker, but right now they are just something to run manually on minifi with a working local broker. I will soon get to start working on dockerized tests, that PR will probably get rid of these.

##########
File path: extensions/librdkafka/tests/ConsumeKafkaTests.cpp
##########
@@ -0,0 +1,590 @@
+
+/**
+ * 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 "../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(), "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 = "Failed to create Kafka producer" + std::string{ errstr.data() };
+      throw std::runtime_error(error_msg);
+    }
+
+    // The last argument is a config here, but it is already owned by the producer. 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> CANCELLED_TRANSACTION        { 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) {
+    reInitialize();
+
+    // Consumer chain
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+    std::shared_ptr<core::Processor> extract_text  = plan_->addProcessor("ExtractText", "extract_text", {success}, false);
+
+    // Set up connections
+    plan_->addConnection(consume_kafka, success, extract_text);
+    extract_text->setAutoTerminatedRelationships({success});
+
+    const auto bool_to_string = [] (const bool b) -> std::string { return b ? "true" : "false"; };
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::SecurityProtocol.getName(), security_protocol);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), topic_names);
+
+    optional_set_property(consume_kafka, ConsumeKafka::TopicNameFormat.getName(), topic_name_format);
+    optional_set_property(consume_kafka, ConsumeKafka::HonorTransactions.getName(), honor_transactions | utils::map(bool_to_string));
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+    optional_set_property(consume_kafka, ConsumeKafka::OffsetReset.getName(), offset_reset);
+    optional_set_property(consume_kafka, ConsumeKafka::KeyAttributeEncoding.getName(), key_attribute_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageDemarcator.getName(), message_demarcator);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageHeaderEncoding.getName(), message_header_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::HeadersToAddAsAttributes.getName(), headers_to_add_as_attributes);
+    optional_set_property(consume_kafka, ConsumeKafka::DuplicateHeaderHandling.getName(), duplicate_header_handling);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    plan_->setProperty(extract_text, ExtractText::Attribute.getName(), ATTRIBUTE_FOR_CAPTURING_CONTENT);
+
+    if (!expect_config_valid) {
+      REQUIRE_THROWS(plan_->scheduleProcessor(consume_kafka));
+      return;
+    } else {
+      plan_->scheduleProcessors();
+    }
+
+    std::unique_ptr<rd_kafka_conf_t, utils::rd_kafka_conf_deleter> conf_;
+    std::unique_ptr<rd_kafka_t, utils::rd_kafka_consumer_deleter> consumer_;
+
+    const bool is_transactional = std::count(transaction_events.cbegin(), transaction_events.cend(), KafkaTestProducer::PublishEvent::TRANSACTION_START);
+    const bool transactions_committed = transaction_events.back() == KafkaTestProducer::PublishEvent::TRANSACTION_COMMIT;
+
+    KafkaTestProducer producer(kafka_brokers, PRODUCER_TOPIC, is_transactional);
+    producer.publish_messages_to_topic(messages_on_topic, TEST_MESSAGE_KEY, transaction_events, message_headers, message_header_encoding);
+
+
+    std::vector<std::shared_ptr<core::FlowFile>> flow_files_produced;
+    for (std::size_t num_expected_messages_processed = 0; num_expected_messages_processed < messages_on_topic.size(); num_expected_messages_processed += std::stoi(max_poll_records.value_or("1"))) {
+      plan_->increment_location();
+      if ((honor_transactions && false == honor_transactions.value()) || (is_transactional && !transactions_committed)) {
+        INFO("Non-committed messages received.");
+        REQUIRE(false == plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+        return;
+      }
+      {
+        SCOPED_INFO("ConsumeKafka timed out when waiting to receive the message published to the kafka broker.");
+        REQUIRE(plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+      }
+      std::size_t num_flow_files_produced = plan_->getNumFlowFileProducedByCurrentProcessor();
+      plan_->increment_location();
+      for (std::size_t times_extract_text_run = 0; times_extract_text_run < num_flow_files_produced; ++times_extract_text_run) {
+        plan_->runCurrentProcessor();  // ExtractText
+        std::shared_ptr<core::FlowFile> flow_file = plan_->getFlowFileProducedByCurrentProcessor();
+        for (const auto& exp_header : expect_header_attributes) {
+          SCOPED_INFO("ConsumeKafka did not produce the expected flowfile attribute from message header: " << exp_header.first << ".");
+          const auto header_attr_opt = flow_file->getAttribute(exp_header.first);
+          REQUIRE(header_attr_opt);
+          REQUIRE(exp_header.second == header_attr_opt.value().get());
+        }
+        {
+          SCOPED_INFO("Message key is missing or incorrect (potential encoding mismatch).");
+          REQUIRE(TEST_MESSAGE_KEY == decode_key(flow_file->getAttribute(ConsumeKafka::KAFKA_MESSAGE_KEY_ATTR).value().get(), key_attribute_encoding));
+          REQUIRE("1" == flow_file->getAttribute(ConsumeKafka::KAFKA_COUNT_ATTR).value().get());
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_OFFSET_ATTR));
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_PARTITION_ATTR));
+          REQUIRE(PRODUCER_TOPIC == flow_file->getAttribute(ConsumeKafka::KAFKA_TOPIC_ATTR).value().get());
+        }
+        flow_files_produced.emplace_back(std::move(flow_file));
+      }
+      plan_->reset_location();
+    }
+
+    const auto contentOrderOfFlowFile = [&] (const std::shared_ptr<core::FlowFile>& lhs, const std::shared_ptr<core::FlowFile>& rhs) {
+      return lhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() < rhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get();
+    };
+    {
+      SCOPED_INFO("The flowfiles generated by ConsumeKafka are invalid (probably nullptr).");
+      REQUIRE_NOTHROW(std::sort(flow_files_produced.begin(), flow_files_produced.end(), contentOrderOfFlowFile));
+    }
+    std::vector<std::string> sorted_split_messages = sort_and_split_messages(messages_on_topic, message_demarcator);
+    const auto flow_file_content_matches_message = [&] (const std::shared_ptr<core::FlowFile>& flowfile, const std::string message) {
+      return flowfile->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() == message;
+    };
+
+    logger_->log_debug("************");
+    std::string expected = "Expected: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      expected += sorted_split_messages[i] + ", ";
+    }
+    std::string   actual = "  Actual: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      actual += flow_files_produced[i]->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() + ", ";
+    }
+    logger_->log_debug("%s", expected.c_str());
+    logger_->log_debug("%s", actual.c_str());
+    logger_->log_debug("************");
+
+    INFO("The messages received by ConsumeKafka do not match those published");
+    REQUIRE(std::equal(flow_files_produced.begin(), flow_files_produced.end(), sorted_split_messages.begin(), flow_file_content_matches_message));
+  }
+};
+
+class ConsumeKafkaContinuousPublishingTest : public ConsumeKafkaTest {
+ public:
+  ConsumeKafkaContinuousPublishingTest() : ConsumeKafkaTest() {}
+  virtual ~ConsumeKafkaContinuousPublishingTest() {
+    logTestController_.reset();
+  }
+
+  void single_consumer_with_continuous_message_producing(
+      const uint64_t msg_periodicity_ms,
+      const std::string& kafka_brokers,
+      const optional<std::string>& group_id,
+      const optional<std::string>& max_poll_records,
+      const optional<std::string>& max_poll_time,
+      const optional<std::string>& session_timeout) {
+    reInitialize();
+
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+
+    plan_->setProperty(consume_kafka, "allow.auto.create.topics", "true", true);  // Seems like the topic tests work without this
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), PRODUCER_TOPIC);
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    consume_kafka->setAutoTerminatedRelationships({success});
+
+    KafkaTestProducer producer("localhost:9092", PRODUCER_TOPIC, /* transactional = */ false);
+
+    std::atomic_bool producer_loop_stop{ false };
+    const auto producer_loop = [&] {
+      std::size_t num_messages_sent = 0;  // When on C++14 make this an initialized lambda capture

Review comment:
       Good catch!

##########
File path: libminifi/include/core/FlowFile.h
##########
@@ -134,7 +135,9 @@ class FlowFile : public CoreComponent, public ReferenceContainer {
    * @param value value to set
    * @return result of finding key
    */
-  bool getAttribute(std::string key, std::string& value) const;
+  bool getAttribute(const std::string& key, std::string& value) const;
+
+  utils::optional<std::reference_wrapper<const std::string>> getAttribute(const std::string& key) const;

Review comment:
       Updated.

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,582 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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());

Review comment:
       Added bolding for `Topic Names`, `Topic Name Format` and `Group ID`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553895379



##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);
+  }
+}
+
+void print_topics_list(std::shared_ptr<logging::Logger> logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger->log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld]",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, const std::shared_ptr<logging::Logger>& logger) {

Review comment:
       Can a const ref on a shared pointer even manipulate lifetime or ownership? I don't mind changing so updated them as requested.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553166725



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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());

Review comment:
       Indeed. Topic names, topic name format and group id are all required properties. I added the required marker to the property building.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] fgerlits commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
fgerlits commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r552700088



##########
File path: extensions/librdkafka/tests/ConsumeKafkaTests.cpp
##########
@@ -0,0 +1,593 @@
+
+/**
+ * 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 "../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(), "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

Review comment:
       typo? consumer -> producer

##########
File path: extensions/librdkafka/tests/ConsumeKafkaTests.cpp
##########
@@ -0,0 +1,593 @@
+
+/**
+ * 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 "../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(), "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());

Review comment:
       `join_pack` joins the strings, it does not do `%s` substitution.  Did you want to use `fmt::format()`?

##########
File path: extensions/librdkafka/tests/ConsumeKafkaTests.cpp
##########
@@ -0,0 +1,593 @@
+
+/**
+ * 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 "../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(), "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 };

Review comment:
       this name is misleading as it does not really do a commit; I would call it `CANCELLED_TRANSACTION`

##########
File path: libminifi/include/core/TypedValues.h
##########
@@ -106,6 +106,8 @@ class TimePeriodValue : public TransformableValue, public state::response::UInt6
   static bool StringToTime(std::string input, uint64_t &output, TimeUnit &timeunit) {
     return utils::internal::StringToTime(input, output, timeunit);
   }
+
+  TimePeriodValue& operator=(const TimePeriodValue& other) = default;

Review comment:
       Why is this needed?  I think the compiler would generate it automatically.  Also, by writing this you disable the move constructor.

##########
File path: extensions/librdkafka/tests/ConsumeKafkaTests.cpp
##########
@@ -0,0 +1,593 @@
+
+/**
+ * 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 "../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(), "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) {
+    reInitialize();
+
+    // Consumer chain
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+    std::shared_ptr<core::Processor> extract_text  = plan_->addProcessor("ExtractText", "extract_text", {success}, false);
+
+    // Set up connections
+    plan_->addConnection(consume_kafka, success, extract_text);
+    extract_text->setAutoTerminatedRelationships({success});
+
+    const auto bool_to_string = [] (const bool b) -> std::string { return b ? "true" : "false"; };
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::SecurityProtocol.getName(), security_protocol);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), topic_names);
+
+    optional_set_property(consume_kafka, ConsumeKafka::TopicNameFormat.getName(), topic_name_format);
+    optional_set_property(consume_kafka, ConsumeKafka::HonorTransactions.getName(), honor_transactions | utils::map(bool_to_string));
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+    optional_set_property(consume_kafka, ConsumeKafka::OffsetReset.getName(), offset_reset);
+    optional_set_property(consume_kafka, ConsumeKafka::KeyAttributeEncoding.getName(), key_attribute_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageDemarcator.getName(), message_demarcator);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageHeaderEncoding.getName(), message_header_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::HeadersToAddAsAttributes.getName(), headers_to_add_as_attributes);
+    optional_set_property(consume_kafka, ConsumeKafka::DuplicateHeaderHandling.getName(), duplicate_header_handling);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    plan_->setProperty(extract_text, ExtractText::Attribute.getName(), ATTRIBUTE_FOR_CAPTURING_CONTENT);
+
+    if (!expect_config_valid) {
+      const auto& message = messages_on_topic.front();
+      REQUIRE_THROWS(plan_->scheduleProcessor(consume_kafka));
+      return;
+    } else {
+      plan_->scheduleProcessors();
+    }
+
+    std::unique_ptr<rd_kafka_conf_t, utils::rd_kafka_conf_deleter> conf_;
+    std::unique_ptr<rd_kafka_t, utils::rd_kafka_consumer_deleter> consumer_;
+
+    const bool is_transactional = std::count(transaction_events.cbegin(), transaction_events.cend(), KafkaTestProducer::PublishEvent::TRANSACTION_START);
+    const bool transactions_committed = transaction_events.back() == KafkaTestProducer::PublishEvent::TRANSACTION_COMMIT;
+
+    KafkaTestProducer producer(kafka_brokers, PRODUCER_TOPIC, is_transactional);
+    producer.publish_messages_to_topic(messages_on_topic, TEST_MESSAGE_KEY, transaction_events, message_headers, message_header_encoding);
+
+
+    std::vector<std::shared_ptr<core::FlowFile>> flow_files_produced;
+    for (std::size_t num_expected_messages_processed = 0; num_expected_messages_processed < messages_on_topic.size(); num_expected_messages_processed += std::stoi(max_poll_records.value_or("1"))) {
+      plan_->increment_location();
+      if ((honor_transactions && false == honor_transactions.value()) || (is_transactional && !transactions_committed)) {
+        INFO("Non-committed messages received.");
+        REQUIRE(false == plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+        return;
+      }
+      {
+        SCOPED_INFO("ConsumeKafka timed out when waiting to receive the message published to the kafka broker.");
+        REQUIRE(plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+      }
+      std::size_t num_flow_files_produced = plan_->getNumFlowFileProducedByCurrentProcessor();
+      plan_->increment_location();
+      for (std::size_t times_extract_text_run = 0; times_extract_text_run < num_flow_files_produced; ++times_extract_text_run) {
+        plan_->runCurrentProcessor();  // ExtractText
+        std::shared_ptr<core::FlowFile> flow_file = plan_->getFlowFileProducedByCurrentProcessor();
+        for (const auto& exp_header : expect_header_attributes) {
+          SCOPED_INFO("ConsumeKafka did not produce the expected flowfile attribute from message header: " << exp_header.first << ".");
+          const auto header_attr_opt = flow_file->getAttribute(exp_header.first);
+          REQUIRE(header_attr_opt);
+          REQUIRE(exp_header.second == header_attr_opt.value().get());
+        }
+        {
+          SCOPED_INFO("Message key is missing or incorrect (potential encoding mismatch).");
+          REQUIRE(TEST_MESSAGE_KEY == decode_key(flow_file->getAttribute(ConsumeKafka::KAFKA_MESSAGE_KEY_ATTR).value().get(), key_attribute_encoding));
+          REQUIRE("1" == flow_file->getAttribute(ConsumeKafka::KAFKA_COUNT_ATTR).value().get());
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_OFFSET_ATTR));
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_PARTITION_ATTR));
+          REQUIRE(PRODUCER_TOPIC == flow_file->getAttribute(ConsumeKafka::KAFKA_TOPIC_ATTR).value().get());
+        }
+        flow_files_produced.emplace_back(std::move(flow_file));
+      }
+      plan_->reset_location();
+    }
+
+    const auto contentOrderOfFlowFile = [&] (const std::shared_ptr<core::FlowFile>& lhs, const std::shared_ptr<core::FlowFile>& rhs) {
+      return lhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() < rhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get();
+    };
+    {
+      SCOPED_INFO("The flowfiles generated by ConsumeKafka are invalid (probably nullptr).");
+      CHECK_NOTHROW(std::sort(flow_files_produced.begin(), flow_files_produced.end(), contentOrderOfFlowFile));

Review comment:
       Does it make sense to continue after `std::sort` throws?  If not, then `REQUIRE_NOTHROW` would be better.

##########
File path: extensions/librdkafka/tests/ConsumeKafkaTests.cpp
##########
@@ -0,0 +1,593 @@
+
+/**
+ * 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 "../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(), "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) {
+    reInitialize();
+
+    // Consumer chain
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+    std::shared_ptr<core::Processor> extract_text  = plan_->addProcessor("ExtractText", "extract_text", {success}, false);
+
+    // Set up connections
+    plan_->addConnection(consume_kafka, success, extract_text);
+    extract_text->setAutoTerminatedRelationships({success});
+
+    const auto bool_to_string = [] (const bool b) -> std::string { return b ? "true" : "false"; };
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::SecurityProtocol.getName(), security_protocol);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), topic_names);
+
+    optional_set_property(consume_kafka, ConsumeKafka::TopicNameFormat.getName(), topic_name_format);
+    optional_set_property(consume_kafka, ConsumeKafka::HonorTransactions.getName(), honor_transactions | utils::map(bool_to_string));
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+    optional_set_property(consume_kafka, ConsumeKafka::OffsetReset.getName(), offset_reset);
+    optional_set_property(consume_kafka, ConsumeKafka::KeyAttributeEncoding.getName(), key_attribute_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageDemarcator.getName(), message_demarcator);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageHeaderEncoding.getName(), message_header_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::HeadersToAddAsAttributes.getName(), headers_to_add_as_attributes);
+    optional_set_property(consume_kafka, ConsumeKafka::DuplicateHeaderHandling.getName(), duplicate_header_handling);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    plan_->setProperty(extract_text, ExtractText::Attribute.getName(), ATTRIBUTE_FOR_CAPTURING_CONTENT);
+
+    if (!expect_config_valid) {
+      const auto& message = messages_on_topic.front();
+      REQUIRE_THROWS(plan_->scheduleProcessor(consume_kafka));
+      return;
+    } else {
+      plan_->scheduleProcessors();
+    }
+
+    std::unique_ptr<rd_kafka_conf_t, utils::rd_kafka_conf_deleter> conf_;
+    std::unique_ptr<rd_kafka_t, utils::rd_kafka_consumer_deleter> consumer_;
+
+    const bool is_transactional = std::count(transaction_events.cbegin(), transaction_events.cend(), KafkaTestProducer::PublishEvent::TRANSACTION_START);
+    const bool transactions_committed = transaction_events.back() == KafkaTestProducer::PublishEvent::TRANSACTION_COMMIT;
+
+    KafkaTestProducer producer(kafka_brokers, PRODUCER_TOPIC, is_transactional);
+    producer.publish_messages_to_topic(messages_on_topic, TEST_MESSAGE_KEY, transaction_events, message_headers, message_header_encoding);
+
+
+    std::vector<std::shared_ptr<core::FlowFile>> flow_files_produced;
+    for (std::size_t num_expected_messages_processed = 0; num_expected_messages_processed < messages_on_topic.size(); num_expected_messages_processed += std::stoi(max_poll_records.value_or("1"))) {
+      plan_->increment_location();
+      if ((honor_transactions && false == honor_transactions.value()) || (is_transactional && !transactions_committed)) {
+        INFO("Non-committed messages received.");
+        REQUIRE(false == plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+        return;
+      }
+      {
+        SCOPED_INFO("ConsumeKafka timed out when waiting to receive the message published to the kafka broker.");
+        REQUIRE(plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+      }
+      std::size_t num_flow_files_produced = plan_->getNumFlowFileProducedByCurrentProcessor();
+      plan_->increment_location();
+      for (std::size_t times_extract_text_run = 0; times_extract_text_run < num_flow_files_produced; ++times_extract_text_run) {
+        plan_->runCurrentProcessor();  // ExtractText
+        std::shared_ptr<core::FlowFile> flow_file = plan_->getFlowFileProducedByCurrentProcessor();
+        for (const auto& exp_header : expect_header_attributes) {
+          SCOPED_INFO("ConsumeKafka did not produce the expected flowfile attribute from message header: " << exp_header.first << ".");
+          const auto header_attr_opt = flow_file->getAttribute(exp_header.first);
+          REQUIRE(header_attr_opt);
+          REQUIRE(exp_header.second == header_attr_opt.value().get());
+        }
+        {
+          SCOPED_INFO("Message key is missing or incorrect (potential encoding mismatch).");
+          REQUIRE(TEST_MESSAGE_KEY == decode_key(flow_file->getAttribute(ConsumeKafka::KAFKA_MESSAGE_KEY_ATTR).value().get(), key_attribute_encoding));
+          REQUIRE("1" == flow_file->getAttribute(ConsumeKafka::KAFKA_COUNT_ATTR).value().get());
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_OFFSET_ATTR));
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_PARTITION_ATTR));
+          REQUIRE(PRODUCER_TOPIC == flow_file->getAttribute(ConsumeKafka::KAFKA_TOPIC_ATTR).value().get());
+        }
+        flow_files_produced.emplace_back(std::move(flow_file));
+      }
+      plan_->reset_location();
+    }
+
+    const auto contentOrderOfFlowFile = [&] (const std::shared_ptr<core::FlowFile>& lhs, const std::shared_ptr<core::FlowFile>& rhs) {
+      return lhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() < rhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get();
+    };
+    {
+      SCOPED_INFO("The flowfiles generated by ConsumeKafka are invalid (probably nullptr).");
+      CHECK_NOTHROW(std::sort(flow_files_produced.begin(), flow_files_produced.end(), contentOrderOfFlowFile));
+    }
+    std::vector<std::string> sorted_split_messages = sort_and_split_messages(messages_on_topic, message_demarcator);
+    const auto flow_file_content_matches_message = [&] (const std::shared_ptr<core::FlowFile>& flowfile, const std::string message) {
+      return flowfile->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() == message;
+    };
+
+    logger_->log_debug("************");
+    std::string expected = "Expected: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      expected += sorted_split_messages[i] + ", ";
+    }
+    std::string   actual = "  Actual: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      actual += flow_files_produced[i]->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() + ", ";
+    }
+    logger_->log_debug("%s", expected.c_str());
+    logger_->log_debug("%s", actual.c_str());
+    logger_->log_debug("************");
+
+    INFO("The messages received by ConsumeKafka do not match those published");
+    REQUIRE(std::equal(flow_files_produced.begin(), flow_files_produced.end(), sorted_split_messages.begin(), flow_file_content_matches_message));
+  }
+};
+
+class ConsumeKafkaContinuousPublishingTest : public ConsumeKafkaTest {
+ public:
+  ConsumeKafkaContinuousPublishingTest() : ConsumeKafkaTest() {}
+  virtual ~ConsumeKafkaContinuousPublishingTest() {
+    logTestController_.reset();
+  }
+
+  void single_consumer_with_continuous_message_producing(
+      const uint64_t msg_periodicity_ms,
+      const std::string& kafka_brokers,
+      const optional<std::string>& group_id,
+      const optional<std::string>& max_poll_records,
+      const optional<std::string>& max_poll_time,
+      const optional<std::string>& session_timeout) {
+    reInitialize();
+
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+
+    plan_->setProperty(consume_kafka, "allow.auto.create.topics", "true", true);  // Seems like the topic tests work without this
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), PRODUCER_TOPIC);
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    consume_kafka->setAutoTerminatedRelationships({success});
+
+    KafkaTestProducer producer("localhost:9092", PRODUCER_TOPIC, /* transactional = */ false);
+
+    std::atomic_bool producer_loop_stop{ false };
+    std::size_t num_messages_sent = 0;  // When on C++14 make this an initialized lambda capture
+    const auto producer_loop = [&] {
+      std::this_thread::sleep_for(std::chrono::milliseconds(100));
+      while (!producer_loop_stop) {
+        producer.publish_messages_to_topic({ "Message after " + std::to_string(msg_periodicity_ms * num_messages_sent) + " ms"}, TEST_MESSAGE_KEY, { PUBLISH }, {}, {});
+        ++num_messages_sent;
+        std::this_thread::sleep_for(std::chrono::milliseconds(msg_periodicity_ms));
+      }
+      return num_messages_sent;
+    };
+
+    plan_->scheduleProcessors();
+
+    const auto get_time_property_ms = [] (const std::string& property_string) {
+      int64_t value;
+      org::apache::nifi::minifi::core::TimeUnit unit;
+      REQUIRE(org::apache::nifi::minifi::core::Property::StringToTime(property_string, value, unit));
+      int64_t value_as_ms;
+      REQUIRE(org::apache::nifi::minifi::core::Property::ConvertTimeUnitToMS(value, unit, value_as_ms));
+      return value_as_ms;
+    };
+
+    std::thread producer_thread(producer_loop);
+    CHECK_NOTHROW(plan_->runNextProcessor());
+    producer_loop_stop = true;
+    producer_thread.join();

Review comment:
       I guess `runNextProcessor()` is wrapped in `CHECK_NOTHROW` to make sure `producer_thread` will be joined?  This is like catching the exception and then carrying on, which is not nice.
   
   I think something like this would be a better solution:
   
   ```suggestion
       {
         std::thread producer_thread(producer_loop);
         auto producer_thread_joiner = gsl::finally([&] {
           producer_loop_stop = true;
           producer_thread.join();
         });
         plan_->runNextProcessor();
       }
   ```

##########
File path: extensions/librdkafka/tests/ConsumeKafkaTests.cpp
##########
@@ -0,0 +1,593 @@
+
+/**
+ * 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 "../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(), "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) {
+    reInitialize();
+
+    // Consumer chain
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+    std::shared_ptr<core::Processor> extract_text  = plan_->addProcessor("ExtractText", "extract_text", {success}, false);
+
+    // Set up connections
+    plan_->addConnection(consume_kafka, success, extract_text);
+    extract_text->setAutoTerminatedRelationships({success});
+
+    const auto bool_to_string = [] (const bool b) -> std::string { return b ? "true" : "false"; };
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::SecurityProtocol.getName(), security_protocol);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), topic_names);
+
+    optional_set_property(consume_kafka, ConsumeKafka::TopicNameFormat.getName(), topic_name_format);
+    optional_set_property(consume_kafka, ConsumeKafka::HonorTransactions.getName(), honor_transactions | utils::map(bool_to_string));
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+    optional_set_property(consume_kafka, ConsumeKafka::OffsetReset.getName(), offset_reset);
+    optional_set_property(consume_kafka, ConsumeKafka::KeyAttributeEncoding.getName(), key_attribute_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageDemarcator.getName(), message_demarcator);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageHeaderEncoding.getName(), message_header_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::HeadersToAddAsAttributes.getName(), headers_to_add_as_attributes);
+    optional_set_property(consume_kafka, ConsumeKafka::DuplicateHeaderHandling.getName(), duplicate_header_handling);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    plan_->setProperty(extract_text, ExtractText::Attribute.getName(), ATTRIBUTE_FOR_CAPTURING_CONTENT);
+
+    if (!expect_config_valid) {
+      const auto& message = messages_on_topic.front();

Review comment:
       unused variable

##########
File path: libminifi/test/TestBase.cpp
##########
@@ -62,78 +63,51 @@ TestPlan::~TestPlan() {
   for (auto& processor : configured_processors_) {
     processor->setScheduledState(core::ScheduledState::STOPPED);
   }
+  for (auto& connection : relationships_) {
+    // This is a patch solving circular references between processors and connections
+    connection->setSource(nullptr);
+    connection->setDestination(nullptr);
+  }
   controller_services_provider_->clearControllerServices();
 }
 
 std::shared_ptr<core::Processor> TestPlan::addProcessor(const std::shared_ptr<core::Processor> &processor, const std::string &name, const std::initializer_list<core::Relationship>& relationships,
-                                                        bool linkToPrevious) {
+    bool linkToPrevious) {
   if (finalized) {
     return nullptr;
   }
   std::lock_guard<std::recursive_mutex> guard(mutex);
-
   utils::Identifier uuid = utils::IdGenerator::getIdGenerator()->generate();
-
   processor->setStreamFactory(stream_factory);
   // initialize the processor
   processor->initialize();
   processor->setFlowIdentifier(flow_version_->getFlowIdentifier());
-
   processor_mapping_[processor->getUUID()] = processor;
-
   if (!linkToPrevious) {
     termination_ = *(relationships.begin());
   } else {
     std::shared_ptr<core::Processor> last = processor_queue_.back();
-
     if (last == nullptr) {
       last = processor;
       termination_ = *(relationships.begin());
     }
-
-    std::stringstream connection_name;
-    connection_name << last->getUUIDStr() << "-to-" << processor->getUUIDStr();
-    logger_->log_info("Creating %s connection for proc %d", connection_name.str(), processor_queue_.size() + 1);
-    std::shared_ptr<minifi::Connection> connection = std::make_shared<minifi::Connection>(flow_repo_, content_repo_, connection_name.str());
-
     for (const auto& relationship : relationships) {
-      connection->addRelationship(relationship);
-    }
-
-    // link the connections so that we can test results at the end for this
-    connection->setSource(last);
-    connection->setDestination(processor);
-
-    connection->setSourceUUID(last->getUUID());
-    connection->setDestinationUUID(processor->getUUID());
-    last->addConnection(connection);
-    if (last != processor) {
-      processor->addConnection(connection);
+      addConnection(last, relationship, processor);

Review comment:
       We used to have a single Connection with multiple Relationships, now we'll have a separate Connection for each Relationship.  What is the reason for this change?

##########
File path: extensions/librdkafka/tests/ConsumeKafkaTests.cpp
##########
@@ -0,0 +1,593 @@
+
+/**
+ * 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 "../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(), "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) {
+    reInitialize();
+
+    // Consumer chain
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+    std::shared_ptr<core::Processor> extract_text  = plan_->addProcessor("ExtractText", "extract_text", {success}, false);
+
+    // Set up connections
+    plan_->addConnection(consume_kafka, success, extract_text);
+    extract_text->setAutoTerminatedRelationships({success});
+
+    const auto bool_to_string = [] (const bool b) -> std::string { return b ? "true" : "false"; };
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::SecurityProtocol.getName(), security_protocol);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), topic_names);
+
+    optional_set_property(consume_kafka, ConsumeKafka::TopicNameFormat.getName(), topic_name_format);
+    optional_set_property(consume_kafka, ConsumeKafka::HonorTransactions.getName(), honor_transactions | utils::map(bool_to_string));
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+    optional_set_property(consume_kafka, ConsumeKafka::OffsetReset.getName(), offset_reset);
+    optional_set_property(consume_kafka, ConsumeKafka::KeyAttributeEncoding.getName(), key_attribute_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageDemarcator.getName(), message_demarcator);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageHeaderEncoding.getName(), message_header_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::HeadersToAddAsAttributes.getName(), headers_to_add_as_attributes);
+    optional_set_property(consume_kafka, ConsumeKafka::DuplicateHeaderHandling.getName(), duplicate_header_handling);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    plan_->setProperty(extract_text, ExtractText::Attribute.getName(), ATTRIBUTE_FOR_CAPTURING_CONTENT);
+
+    if (!expect_config_valid) {
+      const auto& message = messages_on_topic.front();
+      REQUIRE_THROWS(plan_->scheduleProcessor(consume_kafka));
+      return;
+    } else {
+      plan_->scheduleProcessors();
+    }
+
+    std::unique_ptr<rd_kafka_conf_t, utils::rd_kafka_conf_deleter> conf_;
+    std::unique_ptr<rd_kafka_t, utils::rd_kafka_consumer_deleter> consumer_;
+
+    const bool is_transactional = std::count(transaction_events.cbegin(), transaction_events.cend(), KafkaTestProducer::PublishEvent::TRANSACTION_START);
+    const bool transactions_committed = transaction_events.back() == KafkaTestProducer::PublishEvent::TRANSACTION_COMMIT;
+
+    KafkaTestProducer producer(kafka_brokers, PRODUCER_TOPIC, is_transactional);
+    producer.publish_messages_to_topic(messages_on_topic, TEST_MESSAGE_KEY, transaction_events, message_headers, message_header_encoding);
+
+
+    std::vector<std::shared_ptr<core::FlowFile>> flow_files_produced;
+    for (std::size_t num_expected_messages_processed = 0; num_expected_messages_processed < messages_on_topic.size(); num_expected_messages_processed += std::stoi(max_poll_records.value_or("1"))) {
+      plan_->increment_location();
+      if ((honor_transactions && false == honor_transactions.value()) || (is_transactional && !transactions_committed)) {
+        INFO("Non-committed messages received.");
+        REQUIRE(false == plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+        return;
+      }
+      {
+        SCOPED_INFO("ConsumeKafka timed out when waiting to receive the message published to the kafka broker.");
+        REQUIRE(plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+      }
+      std::size_t num_flow_files_produced = plan_->getNumFlowFileProducedByCurrentProcessor();
+      plan_->increment_location();
+      for (std::size_t times_extract_text_run = 0; times_extract_text_run < num_flow_files_produced; ++times_extract_text_run) {
+        plan_->runCurrentProcessor();  // ExtractText
+        std::shared_ptr<core::FlowFile> flow_file = plan_->getFlowFileProducedByCurrentProcessor();
+        for (const auto& exp_header : expect_header_attributes) {
+          SCOPED_INFO("ConsumeKafka did not produce the expected flowfile attribute from message header: " << exp_header.first << ".");
+          const auto header_attr_opt = flow_file->getAttribute(exp_header.first);
+          REQUIRE(header_attr_opt);
+          REQUIRE(exp_header.second == header_attr_opt.value().get());
+        }
+        {
+          SCOPED_INFO("Message key is missing or incorrect (potential encoding mismatch).");
+          REQUIRE(TEST_MESSAGE_KEY == decode_key(flow_file->getAttribute(ConsumeKafka::KAFKA_MESSAGE_KEY_ATTR).value().get(), key_attribute_encoding));
+          REQUIRE("1" == flow_file->getAttribute(ConsumeKafka::KAFKA_COUNT_ATTR).value().get());
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_OFFSET_ATTR));
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_PARTITION_ATTR));
+          REQUIRE(PRODUCER_TOPIC == flow_file->getAttribute(ConsumeKafka::KAFKA_TOPIC_ATTR).value().get());
+        }
+        flow_files_produced.emplace_back(std::move(flow_file));
+      }
+      plan_->reset_location();
+    }
+
+    const auto contentOrderOfFlowFile = [&] (const std::shared_ptr<core::FlowFile>& lhs, const std::shared_ptr<core::FlowFile>& rhs) {
+      return lhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() < rhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get();
+    };
+    {
+      SCOPED_INFO("The flowfiles generated by ConsumeKafka are invalid (probably nullptr).");
+      CHECK_NOTHROW(std::sort(flow_files_produced.begin(), flow_files_produced.end(), contentOrderOfFlowFile));
+    }
+    std::vector<std::string> sorted_split_messages = sort_and_split_messages(messages_on_topic, message_demarcator);
+    const auto flow_file_content_matches_message = [&] (const std::shared_ptr<core::FlowFile>& flowfile, const std::string message) {
+      return flowfile->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() == message;
+    };
+
+    logger_->log_debug("************");
+    std::string expected = "Expected: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      expected += sorted_split_messages[i] + ", ";
+    }
+    std::string   actual = "  Actual: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      actual += flow_files_produced[i]->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() + ", ";
+    }
+    logger_->log_debug("%s", expected.c_str());
+    logger_->log_debug("%s", actual.c_str());
+    logger_->log_debug("************");
+
+    INFO("The messages received by ConsumeKafka do not match those published");
+    REQUIRE(std::equal(flow_files_produced.begin(), flow_files_produced.end(), sorted_split_messages.begin(), flow_file_content_matches_message));
+  }
+};
+
+class ConsumeKafkaContinuousPublishingTest : public ConsumeKafkaTest {
+ public:
+  ConsumeKafkaContinuousPublishingTest() : ConsumeKafkaTest() {}
+  virtual ~ConsumeKafkaContinuousPublishingTest() {
+    logTestController_.reset();
+  }
+
+  void single_consumer_with_continuous_message_producing(
+      const uint64_t msg_periodicity_ms,
+      const std::string& kafka_brokers,
+      const optional<std::string>& group_id,
+      const optional<std::string>& max_poll_records,
+      const optional<std::string>& max_poll_time,
+      const optional<std::string>& session_timeout) {
+    reInitialize();
+
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+
+    plan_->setProperty(consume_kafka, "allow.auto.create.topics", "true", true);  // Seems like the topic tests work without this
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), PRODUCER_TOPIC);
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    consume_kafka->setAutoTerminatedRelationships({success});
+
+    KafkaTestProducer producer("localhost:9092", PRODUCER_TOPIC, /* transactional = */ false);
+
+    std::atomic_bool producer_loop_stop{ false };
+    std::size_t num_messages_sent = 0;  // When on C++14 make this an initialized lambda capture

Review comment:
       could this be a local variable inside `producer_loop`?

##########
File path: libminifi/include/core/FlowFile.h
##########
@@ -134,7 +135,9 @@ class FlowFile : public CoreComponent, public ReferenceContainer {
    * @param value value to set
    * @return result of finding key
    */
-  bool getAttribute(std::string key, std::string& value) const;
+  bool getAttribute(const std::string& key, std::string& value) const;
+
+  utils::optional<std::reference_wrapper<const std::string>> getAttribute(const std::string& key) const;

Review comment:
       I would prefer `optional<string>` as the return type, maybe with another version returning `string*` for when we must avoid a copy for performance reasons.  Both of these would be simpler and more readable, in my opinion.

##########
File path: extensions/librdkafka/tests/ConsumeKafkaTests.cpp
##########
@@ -0,0 +1,593 @@
+
+/**
+ * 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 "../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(), "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) {
+    reInitialize();
+
+    // Consumer chain
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+    std::shared_ptr<core::Processor> extract_text  = plan_->addProcessor("ExtractText", "extract_text", {success}, false);
+
+    // Set up connections
+    plan_->addConnection(consume_kafka, success, extract_text);
+    extract_text->setAutoTerminatedRelationships({success});
+
+    const auto bool_to_string = [] (const bool b) -> std::string { return b ? "true" : "false"; };
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::SecurityProtocol.getName(), security_protocol);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), topic_names);
+
+    optional_set_property(consume_kafka, ConsumeKafka::TopicNameFormat.getName(), topic_name_format);
+    optional_set_property(consume_kafka, ConsumeKafka::HonorTransactions.getName(), honor_transactions | utils::map(bool_to_string));
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+    optional_set_property(consume_kafka, ConsumeKafka::OffsetReset.getName(), offset_reset);
+    optional_set_property(consume_kafka, ConsumeKafka::KeyAttributeEncoding.getName(), key_attribute_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageDemarcator.getName(), message_demarcator);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageHeaderEncoding.getName(), message_header_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::HeadersToAddAsAttributes.getName(), headers_to_add_as_attributes);
+    optional_set_property(consume_kafka, ConsumeKafka::DuplicateHeaderHandling.getName(), duplicate_header_handling);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    plan_->setProperty(extract_text, ExtractText::Attribute.getName(), ATTRIBUTE_FOR_CAPTURING_CONTENT);
+
+    if (!expect_config_valid) {
+      const auto& message = messages_on_topic.front();
+      REQUIRE_THROWS(plan_->scheduleProcessor(consume_kafka));
+      return;
+    } else {
+      plan_->scheduleProcessors();
+    }
+
+    std::unique_ptr<rd_kafka_conf_t, utils::rd_kafka_conf_deleter> conf_;
+    std::unique_ptr<rd_kafka_t, utils::rd_kafka_consumer_deleter> consumer_;
+
+    const bool is_transactional = std::count(transaction_events.cbegin(), transaction_events.cend(), KafkaTestProducer::PublishEvent::TRANSACTION_START);
+    const bool transactions_committed = transaction_events.back() == KafkaTestProducer::PublishEvent::TRANSACTION_COMMIT;
+
+    KafkaTestProducer producer(kafka_brokers, PRODUCER_TOPIC, is_transactional);
+    producer.publish_messages_to_topic(messages_on_topic, TEST_MESSAGE_KEY, transaction_events, message_headers, message_header_encoding);
+
+
+    std::vector<std::shared_ptr<core::FlowFile>> flow_files_produced;
+    for (std::size_t num_expected_messages_processed = 0; num_expected_messages_processed < messages_on_topic.size(); num_expected_messages_processed += std::stoi(max_poll_records.value_or("1"))) {
+      plan_->increment_location();
+      if ((honor_transactions && false == honor_transactions.value()) || (is_transactional && !transactions_committed)) {
+        INFO("Non-committed messages received.");
+        REQUIRE(false == plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+        return;
+      }
+      {
+        SCOPED_INFO("ConsumeKafka timed out when waiting to receive the message published to the kafka broker.");
+        REQUIRE(plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+      }
+      std::size_t num_flow_files_produced = plan_->getNumFlowFileProducedByCurrentProcessor();
+      plan_->increment_location();
+      for (std::size_t times_extract_text_run = 0; times_extract_text_run < num_flow_files_produced; ++times_extract_text_run) {
+        plan_->runCurrentProcessor();  // ExtractText
+        std::shared_ptr<core::FlowFile> flow_file = plan_->getFlowFileProducedByCurrentProcessor();
+        for (const auto& exp_header : expect_header_attributes) {
+          SCOPED_INFO("ConsumeKafka did not produce the expected flowfile attribute from message header: " << exp_header.first << ".");
+          const auto header_attr_opt = flow_file->getAttribute(exp_header.first);
+          REQUIRE(header_attr_opt);
+          REQUIRE(exp_header.second == header_attr_opt.value().get());
+        }
+        {
+          SCOPED_INFO("Message key is missing or incorrect (potential encoding mismatch).");
+          REQUIRE(TEST_MESSAGE_KEY == decode_key(flow_file->getAttribute(ConsumeKafka::KAFKA_MESSAGE_KEY_ATTR).value().get(), key_attribute_encoding));
+          REQUIRE("1" == flow_file->getAttribute(ConsumeKafka::KAFKA_COUNT_ATTR).value().get());
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_OFFSET_ATTR));
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_PARTITION_ATTR));
+          REQUIRE(PRODUCER_TOPIC == flow_file->getAttribute(ConsumeKafka::KAFKA_TOPIC_ATTR).value().get());
+        }
+        flow_files_produced.emplace_back(std::move(flow_file));
+      }
+      plan_->reset_location();
+    }
+
+    const auto contentOrderOfFlowFile = [&] (const std::shared_ptr<core::FlowFile>& lhs, const std::shared_ptr<core::FlowFile>& rhs) {
+      return lhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() < rhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get();
+    };
+    {
+      SCOPED_INFO("The flowfiles generated by ConsumeKafka are invalid (probably nullptr).");
+      CHECK_NOTHROW(std::sort(flow_files_produced.begin(), flow_files_produced.end(), contentOrderOfFlowFile));
+    }
+    std::vector<std::string> sorted_split_messages = sort_and_split_messages(messages_on_topic, message_demarcator);
+    const auto flow_file_content_matches_message = [&] (const std::shared_ptr<core::FlowFile>& flowfile, const std::string message) {
+      return flowfile->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() == message;
+    };
+
+    logger_->log_debug("************");
+    std::string expected = "Expected: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      expected += sorted_split_messages[i] + ", ";
+    }
+    std::string   actual = "  Actual: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      actual += flow_files_produced[i]->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() + ", ";
+    }
+    logger_->log_debug("%s", expected.c_str());
+    logger_->log_debug("%s", actual.c_str());
+    logger_->log_debug("************");
+
+    INFO("The messages received by ConsumeKafka do not match those published");
+    REQUIRE(std::equal(flow_files_produced.begin(), flow_files_produced.end(), sorted_split_messages.begin(), flow_file_content_matches_message));
+  }
+};
+
+class ConsumeKafkaContinuousPublishingTest : public ConsumeKafkaTest {
+ public:
+  ConsumeKafkaContinuousPublishingTest() : ConsumeKafkaTest() {}
+  virtual ~ConsumeKafkaContinuousPublishingTest() {
+    logTestController_.reset();
+  }
+
+  void single_consumer_with_continuous_message_producing(
+      const uint64_t msg_periodicity_ms,
+      const std::string& kafka_brokers,
+      const optional<std::string>& group_id,
+      const optional<std::string>& max_poll_records,
+      const optional<std::string>& max_poll_time,
+      const optional<std::string>& session_timeout) {
+    reInitialize();
+
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+
+    plan_->setProperty(consume_kafka, "allow.auto.create.topics", "true", true);  // Seems like the topic tests work without this
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), PRODUCER_TOPIC);
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    consume_kafka->setAutoTerminatedRelationships({success});
+
+    KafkaTestProducer producer("localhost:9092", PRODUCER_TOPIC, /* transactional = */ false);
+
+    std::atomic_bool producer_loop_stop{ false };
+    std::size_t num_messages_sent = 0;  // When on C++14 make this an initialized lambda capture
+    const auto producer_loop = [&] {
+      std::this_thread::sleep_for(std::chrono::milliseconds(100));
+      while (!producer_loop_stop) {
+        producer.publish_messages_to_topic({ "Message after " + std::to_string(msg_periodicity_ms * num_messages_sent) + " ms"}, TEST_MESSAGE_KEY, { PUBLISH }, {}, {});
+        ++num_messages_sent;
+        std::this_thread::sleep_for(std::chrono::milliseconds(msg_periodicity_ms));
+      }
+      return num_messages_sent;
+    };
+
+    plan_->scheduleProcessors();
+
+    const auto get_time_property_ms = [] (const std::string& property_string) {
+      int64_t value;
+      org::apache::nifi::minifi::core::TimeUnit unit;
+      REQUIRE(org::apache::nifi::minifi::core::Property::StringToTime(property_string, value, unit));
+      int64_t value_as_ms;
+      REQUIRE(org::apache::nifi::minifi::core::Property::ConvertTimeUnitToMS(value, unit, value_as_ms));
+      return value_as_ms;
+    };
+
+    std::thread producer_thread(producer_loop);
+    CHECK_NOTHROW(plan_->runNextProcessor());
+    producer_loop_stop = true;
+    producer_thread.join();
+
+    std::size_t num_flow_files_produced = plan_->getNumFlowFileProducedByCurrentProcessor();
+
+    const uint64_t max_poll_time_ms = get_time_property_ms(max_poll_time.value_or(ConsumeKafka::DEFAULT_MAX_POLL_TIME));
+    const uint64_t max_poll_records_value = max_poll_records ? std::stoi(max_poll_records.value()) : ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+    const uint64_t exp_lower_bound = std::min(max_poll_time_ms / msg_periodicity_ms - 2, max_poll_records_value);
+    const uint64_t exp_upper_bound = std::min(max_poll_time_ms / msg_periodicity_ms + 2, max_poll_records_value);
+    logger_->log_debug("Max poll time: %d, Max poll records: %d, Exp. flowfiles produced: (min: %d, max: %d), actual: %d",
+        max_poll_time_ms, max_poll_records_value, exp_lower_bound, exp_upper_bound, num_flow_files_produced);
+
+    REQUIRE(exp_lower_bound <= num_flow_files_produced);
+    REQUIRE(num_flow_files_produced <= exp_upper_bound);
+  }
+};
+
+const std::string ConsumeKafkaTest::TEST_FILE_NAME_POSTFIX{ "target_kafka_message.txt" };
+const std::string ConsumeKafkaTest::TEST_MESSAGE_KEY{ "consume_kafka_test_key" };
+const std::string ConsumeKafkaTest::PRODUCER_TOPIC{ "ConsumeKafkaTest" };
+const std::string ConsumeKafkaTest::ATTRIBUTE_FOR_CAPTURING_CONTENT{ "flowfile_content" };
+const std::chrono::seconds ConsumeKafkaTest::MAX_CONSUMEKAFKA_POLL_TIME_SECONDS{ 5 };
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "ConsumeKafka parses and uses kafka topics.", "[ConsumeKafka][Kafka][Topic]") {
+  auto run_tests = [&] (const std::vector<std::string>& messages_on_topic, const std::string& topic_names, const optional<std::string>& topic_name_format) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", topic_names, topic_name_format, {}, "test_group_id", {}, {}, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({ "Ulysses",              "James Joyce"         }, "ConsumeKafkaTest",         {});
+  run_tests({ "The Great Gatsby",     "F. Scott Fitzgerald" }, "ConsumeKafkaTest",         ConsumeKafka::TOPIC_FORMAT_NAMES);
+  run_tests({ "War and Peace",        "Lev Tolstoy"         }, "a,b,c,ConsumeKafkaTest,d", ConsumeKafka::TOPIC_FORMAT_NAMES);
+  run_tests({ "Nineteen Eighty Four", "George Orwell"       }, "ConsumeKafkaTest",         ConsumeKafka::TOPIC_FORMAT_PATTERNS);
+  run_tests({ "Hamlet",               "William Shakespeare" }, "Cons[emu]*KafkaTest",      ConsumeKafka::TOPIC_FORMAT_PATTERNS);
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Offsets are reset to the latest when a consumer starts with non-processed messages.", "[ConsumeKafka][Kafka][OffsetReset]") {
+  auto run_tests = [&] (
+      const std::vector<std::string>& messages_on_topic,
+      const std::vector<KafkaTestProducer::PublishEvent>& transaction_events) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, transaction_events, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  KafkaTestProducer producer("localhost:9092", PRODUCER_TOPIC, false);
+  producer.publish_messages_to_topic({"Dummy messages", "that should be ignored", "due to offset reset on ConsumeKafka startup"}, TEST_MESSAGE_KEY, {PUBLISH, PUBLISH, PUBLISH}, {}, {});
+  run_tests({"Brave New World",  "Aldous Huxley"}, NON_TRANSACTIONAL_MESSAGES);
+  producer.publish_messages_to_topic({"Dummy messages", "that should be ignored", "due to offset reset on ConsumeKafka startup"}, TEST_MESSAGE_KEY, {PUBLISH, PUBLISH, PUBLISH}, {}, {});
+  run_tests({"Call of the Wild", "Jack London"}, NON_TRANSACTIONAL_MESSAGES);
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Key attribute is encoded according to the \"Key Attribute Encoding\" property.", "[ConsumeKafka][Kafka][KeyAttributeEncoding]") {
+  auto run_tests = [&] (const std::vector<std::string>& messages_on_topic, const optional<std::string>& key_attribute_encoding) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, key_attribute_encoding, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+
+  run_tests({ "The Odyssey",          "Ὅμηρος"                        }, {});
+  run_tests({ "Lolita",               "Владимир Владимирович Набоков" }, "utf-8");
+  run_tests({ "Crime and Punishment", "Фёдор Михайлович Достоевский"  }, "hex");
+  run_tests({ "Paradise Lost",        "John Milton"                   }, "hEX");
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Transactional behaviour is supported.", "[ConsumeKafka][Kafka][Transaction]") {
+  auto run_tests = [&] (const std::vector<std::string>& messages_on_topic, const std::vector<KafkaTestProducer::PublishEvent>& transaction_events, const optional<bool>& honor_transactions) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, transaction_events, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({  "Pride and Prejudice", "Jane Austen"      }, SINGLE_COMMITTED_TRANSACTION, {});
+  run_tests({                 "Dune", "Frank Herbert"    },    TWO_SEPARATE_TRANSACTIONS, {});
+  run_tests({      "The Black Sheep", "Honore De Balzac" },    NON_COMMITTED_TRANSACTION, {});
+  run_tests({     "Gospel of Thomas"                     },            COMMIT_AND_CANCEL, {});
+  run_tests({ "Operation Dark Heart"                     },            COMMIT_AND_CANCEL, true);
+  run_tests({               "Brexit"                     },            COMMIT_AND_CANCEL, false);
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Headers on consumed Kafka messages are extracted into attributes if requested on ConsumeKafka.", "[ConsumeKafka][Kafka][Batching][Headers]") {
+  auto run_tests = [&] (
+      const std::vector<std::string>& messages_on_topic,
+      const std::vector<std::pair<std::string, std::string>>& expect_header_attributes,
+      const std::vector<std::pair<std::string, std::string>>& message_headers,
+      const optional<std::string>& headers_to_add_as_attributes,
+      const optional<std::string>& duplicate_header_handling) {
+    single_consumer_with_plain_text_test(true, false, expect_header_attributes, messages_on_topic, NON_TRANSACTIONAL_MESSAGES, message_headers, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, headers_to_add_as_attributes, duplicate_header_handling, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({             "Homeland",   "R. A. Salvatore"},                                      {},             {{{"Contains dark elves"}, {"Yes"}}},         {},                    {});
+  run_tests({             "Magician",  "Raymond E. Feist"},               {{{"Rating"}, {"10/10"}}},                        {{{"Rating"}, {"10/10"}}}, {"Rating"},                    {});
+  run_tests({             "Mistborn", "Brandon Sanderson"},               {{{"Metal"}, {"Copper"}}}, {{{"Metal"}, {"Copper"}}, {{"Metal"}, {"Iron"}}},  {"Metal"},            KEEP_FIRST);
+  run_tests({             "Mistborn", "Brandon Sanderson"},                 {{{"Metal"}, {"Iron"}}}, {{{"Metal"}, {"Copper"}}, {{"Metal"}, {"Iron"}}},  {"Metal"},           KEEP_LATEST);
+  run_tests({             "Mistborn", "Brandon Sanderson"},         {{{"Metal"}, {"Copper, Iron"}}}, {{{"Metal"}, {"Copper"}}, {{"Metal"}, {"Iron"}}},  {"Metal"}, COMMA_SEPARATED_MERGE);
+  run_tests({"The Lord of the Rings",  "J. R. R. Tolkien"}, {{{"Parts"}, {"First, second, third"}}},          {{{"Parts"}, {"First, second, third"}}},  {"Parts"},                    {});
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Messages are separated into multiple flowfiles if the message demarcator is present in the message.", "[ConsumeKafka][Kafka][MessageDemarcator]") {
+  auto run_tests = [&] (
+      const std::vector<std::string>& messages_on_topic,
+      const optional<std::string>& message_demarcator) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, message_demarcator, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({"Barbapapa", "Anette Tison and Talus Taylor"}, "a");
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "The maximum poll records allows ConsumeKafka to combine multiple messages into a single flowfile.", "[ConsumeKafka][Kafka][Batching][MaxPollRecords]") {
+  auto run_tests = [&] (
+      const std::vector<std::string>& messages_on_topic,
+      const std::vector<KafkaTestProducer::PublishEvent>& transaction_events,
+      const optional<std::string>& max_poll_records) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, transaction_events, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, {}, {}, max_poll_records, "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({"The Count of Monte Cristo", "Alexandre Dumas"}, NON_TRANSACTIONAL_MESSAGES, "2");
+
+  const std::vector<std::string> content {
+      "Make const member functions thread safe",
+      "Understand special member function generation",
+      "Use std::unique_ptr for exclusive-ownership resource management",
+      "Use std::shared_ptr for shared-ownership resource management",
+      "Use std::weak_ptr for std::shared_ptr-like pointers that can dangle",
+      "Prefer std::make_unique and std::make_shared to direct use of new",
+      "When using the Pimpl Idiom, define special member functions inthe implementation file",
+      "Understand std::move and std::forward",
+      "Distinguish universal references from rvalue references",
+      "Use std::move on rvalue references, std::forward on universal references",
+      "Avoid overloading on universal references",
+      "Familiarize yourself with alternatives to overloading on universal references",
+      "Understand reference collapsing",
+      "Assume that move operations are not present, not cheap, and not used",
+      "Familiarize yourself with perfect forwarding failure cases",
+      "Avoid default capture modes",
+      "Use init capture to move objects into closures",
+      "Use decltype on auto&& parameters to std::forward them",
+      "Prefer lambdas to std::bind",
+      "Prefer task-based programming to thread-based" };
+  const std::vector<KafkaTestProducer::PublishEvent> transaction_events(content.size(), PUBLISH);
+  run_tests(content, transaction_events, "5");
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Non-plain text security context throws scheduling exceptions.", "[ConsumeKafka][Kafka][SecurityProtocol]") {
+  single_consumer_with_plain_text_test(false, false, {}, { "Miyamoto Musashi", "Eiji Yoshikawa" }, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", ConsumeKafka::SECURITY_PROTOCOL_SSL, "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Acceptable values for message header and key attribute encoding are \"UTF-8\" and \"hex\".", "[ConsumeKafka][Kafka][InvalidEncoding]") {
+  single_consumer_with_plain_text_test(false, false, {}, {                           "Shogun", "James Clavell" }, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, "UTF-32", {},       {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  single_consumer_with_plain_text_test(false, false, {}, { "Alice's Adventures in Wonderland", "Lewis Carroll" }, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {},       {}, {}, "UTF-32", {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+}
+
+TEST_CASE_METHOD(ConsumeKafkaContinuousPublishingTest, "ConsumeKafka can spend no more time polling than allowed in the maximum poll time property.", "[ConsumeKafka][Kafka][Batching][MaxPollTime]") {
+  auto run_tests = [&] (
+      const uint64_t msg_periodicity_ms,
+      const optional<std::string>& max_poll_records,
+      const optional<std::string>& max_poll_time,
+      const optional<std::string>& session_timeout) {
+    single_consumer_with_continuous_message_producing(msg_periodicity_ms, "localhost:9092", "test_group_id", max_poll_records, max_poll_time, session_timeout);
+  };
+  // For some reason, a session time-out of a few seconds does not work at all, 10 seconds seems to be stable
+  run_tests(300, "20", "3 seconds", "10000 ms");

Review comment:
       Is this going to add 10 seconds to the unit tests' running time?  I don't like that.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
lordgamez commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r534990092



##########
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:
       I see I suppose you are right in this case it is explicit what parameters are used as they are fixed and it can be set in a single line. Let's keep it as it is, I'm not against it. The only downside I see is that because of the long list of string parameters it could be easy to mix them up when calling, but I suppose it can be easily realized after the test fails.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553344804



##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);
+  }
+}
+
+void print_topics_list(std::shared_ptr<logging::Logger> logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger->log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld]",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, const std::shared_ptr<logging::Logger>& logger) {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    const std::string error_msg = "ConsumeKafka: received error message from broker. Librdkafka error msg: " + std::string(rd_kafka_err2str(rkmessage->err));
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, error_msg);
+  }
+  std::string topicName = rd_kafka_topic_name(rkmessage->rkt);
+  std::string message(reinterpret_cast<char*>(rkmessage->payload), rkmessage->len);
+  const char* key = reinterpret_cast<const char*>(rkmessage->key);
+  const std::size_t key_len = rkmessage->key_len;
+  rd_kafka_timestamp_type_t tstype;
+  int64_t timestamp;
+  timestamp = rd_kafka_message_timestamp(rkmessage, &tstype);
+  const char *tsname = "?";
+  if (tstype != RD_KAFKA_TIMESTAMP_NOT_AVAILABLE) {
+    if (tstype == RD_KAFKA_TIMESTAMP_CREATE_TIME) {
+      tsname = "create time";
+    } else if (tstype == RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME) {
+      tsname = "log append time";
+    }
+  }
+  const int64_t seconds_since_timestamp = timestamp ? static_cast<int64_t>(time(NULL)) - static_cast<int64_t>(timestamp / 1000) : 0;

Review comment:
       
   
   Is there a reason for printing a relative timestamp instead of the actual absolute one? It takes <= 1 sec of being on the screen (or in the log) for a relative timestamp to become outdated.
   




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553402707



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,562 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);

Review comment:
       Good catch, updated.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r567639924



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,579 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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.")

Review comment:
       The property was taken from the NiFi property description, it is true that we do not do any bundling, and our behaviour is different. Will delete this part of the property description and reword the rest.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] arpadboda closed pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
arpadboda closed pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940


   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r560859704



##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,117 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error: " + error_msg);
+  }
+}
+
+void print_topics_list(logging::Logger& logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger.log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld",

Review comment:
       Wow, you really have thoroughly reviewed this PR, checking even debug logged types. Good catch!




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r534256571



##########
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:
       This is verbatim from nifi, but will rephrase.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] fgerlits commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
fgerlits commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r554934213



##########
File path: extensions/librdkafka/docker_tests/CMakeLists.txt
##########
@@ -0,0 +1,36 @@
+#

Review comment:
       I still think that the future PR in which you dockerize the tests would be a better place for this CMakeLists file, and having a CMakeLists file in a directory by itself is strange, but since this is temporary, I'm OK with it.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r560850272



##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,117 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error: " + error_msg);
+  }
+}
+
+void print_topics_list(logging::Logger& logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger.log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, logging::Logger& logger) {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    const std::string error_msg = "ConsumeKafka: received error message from broker. Librdkafka error msg: " + std::string(rd_kafka_err2str(rkmessage->err));
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, error_msg);
+  }
+  std::string topicName = rd_kafka_topic_name(rkmessage->rkt);
+  std::string message(reinterpret_cast<char*>(rkmessage->payload), rkmessage->len);
+  const char* key = reinterpret_cast<const char*>(rkmessage->key);
+  const std::size_t key_len = rkmessage->key_len;
+  rd_kafka_timestamp_type_t tstype;
+  int64_t timestamp;
+  timestamp = rd_kafka_message_timestamp(rkmessage, &tstype);
+  const char *tsname = "?";
+  if (tstype != RD_KAFKA_TIMESTAMP_NOT_AVAILABLE) {
+    if (tstype == RD_KAFKA_TIMESTAMP_CREATE_TIME) {
+      tsname = "create time";
+    } else if (tstype == RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME) {
+      tsname = "log append time";
+    }
+  }

Review comment:
       Updated with extracting the timestamp serialization.

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,117 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error: " + error_msg);
+  }
+}
+
+void print_topics_list(logging::Logger& logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger.log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, logging::Logger& logger) {

Review comment:
       Extracted serializations.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r577651674



##########
File path: extensions/librdkafka/ConsumeKafka.h
##########
@@ -124,6 +124,17 @@ class ConsumeKafka : public core::Processor {
   // Initialize, overwrite by NiFi RetryFlowFile
   void initialize() override;
 
+  class WriteCallback : public OutputStreamCallback {

Review comment:
       Does it need to be public? It's only used inside the class IIRC




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r534937049



##########
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:
       Good catch!




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r534960543



##########
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:
       While this is a very long list of potential arguments to be tested, I deliberately implemented the tests so that each of them fixes all parameters, that are tested in a given testcase, so that they are easy to read.  I prefer this approach to one with a default constructed list of arguments, as it is less likely to produce superfluous tests (changing one argument takes one line in case of an argument struct).




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553229661



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  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() const {
+  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() const {
+  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) const {
+  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) const {
+  // 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) const {
+  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;
+  }
+};
+
+void ConsumeKafka::add_kafka_attributes_to_flowfile(std::shared_ptr<FlowFileRecord>& flow_file, const rd_kafka_message_t* message) const {
+  // 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, 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));
+}
+
+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) const {
+  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 {};

Review comment:
       Updated this line to `continue`. Doing the same thing with line 515 is unsafe, because due to message demarcation, we can be halfway through a message when the failure happens.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r567663765



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,579 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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 */) {

Review comment:
       Updated.

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,579 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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) {

Review comment:
       Updated.

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,579 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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:
       Updated.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553912615



##########
File path: extensions/librdkafka/tests/ConsumeKafkaTests.cpp
##########
@@ -0,0 +1,590 @@
+
+/**
+ * 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 "../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(), "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 = "Failed to create Kafka producer" + std::string{ errstr.data() };
+      throw std::runtime_error(error_msg);
+    }
+
+    // The last argument is a config here, but it is already owned by the producer. 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> CANCELLED_TRANSACTION        { 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) {
+    reInitialize();
+
+    // Consumer chain
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+    std::shared_ptr<core::Processor> extract_text  = plan_->addProcessor("ExtractText", "extract_text", {success}, false);
+
+    // Set up connections
+    plan_->addConnection(consume_kafka, success, extract_text);
+    extract_text->setAutoTerminatedRelationships({success});
+
+    const auto bool_to_string = [] (const bool b) -> std::string { return b ? "true" : "false"; };
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::SecurityProtocol.getName(), security_protocol);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), topic_names);
+
+    optional_set_property(consume_kafka, ConsumeKafka::TopicNameFormat.getName(), topic_name_format);
+    optional_set_property(consume_kafka, ConsumeKafka::HonorTransactions.getName(), honor_transactions | utils::map(bool_to_string));
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+    optional_set_property(consume_kafka, ConsumeKafka::OffsetReset.getName(), offset_reset);
+    optional_set_property(consume_kafka, ConsumeKafka::KeyAttributeEncoding.getName(), key_attribute_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageDemarcator.getName(), message_demarcator);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageHeaderEncoding.getName(), message_header_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::HeadersToAddAsAttributes.getName(), headers_to_add_as_attributes);
+    optional_set_property(consume_kafka, ConsumeKafka::DuplicateHeaderHandling.getName(), duplicate_header_handling);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    plan_->setProperty(extract_text, ExtractText::Attribute.getName(), ATTRIBUTE_FOR_CAPTURING_CONTENT);
+
+    if (!expect_config_valid) {
+      REQUIRE_THROWS(plan_->scheduleProcessor(consume_kafka));
+      return;
+    } else {
+      plan_->scheduleProcessors();
+    }
+
+    std::unique_ptr<rd_kafka_conf_t, utils::rd_kafka_conf_deleter> conf_;
+    std::unique_ptr<rd_kafka_t, utils::rd_kafka_consumer_deleter> consumer_;
+
+    const bool is_transactional = std::count(transaction_events.cbegin(), transaction_events.cend(), KafkaTestProducer::PublishEvent::TRANSACTION_START);
+    const bool transactions_committed = transaction_events.back() == KafkaTestProducer::PublishEvent::TRANSACTION_COMMIT;
+
+    KafkaTestProducer producer(kafka_brokers, PRODUCER_TOPIC, is_transactional);
+    producer.publish_messages_to_topic(messages_on_topic, TEST_MESSAGE_KEY, transaction_events, message_headers, message_header_encoding);
+
+
+    std::vector<std::shared_ptr<core::FlowFile>> flow_files_produced;
+    for (std::size_t num_expected_messages_processed = 0; num_expected_messages_processed < messages_on_topic.size(); num_expected_messages_processed += std::stoi(max_poll_records.value_or("1"))) {
+      plan_->increment_location();
+      if ((honor_transactions && false == honor_transactions.value()) || (is_transactional && !transactions_committed)) {
+        INFO("Non-committed messages received.");
+        REQUIRE(false == plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+        return;
+      }
+      {
+        SCOPED_INFO("ConsumeKafka timed out when waiting to receive the message published to the kafka broker.");
+        REQUIRE(plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+      }
+      std::size_t num_flow_files_produced = plan_->getNumFlowFileProducedByCurrentProcessor();
+      plan_->increment_location();
+      for (std::size_t times_extract_text_run = 0; times_extract_text_run < num_flow_files_produced; ++times_extract_text_run) {
+        plan_->runCurrentProcessor();  // ExtractText
+        std::shared_ptr<core::FlowFile> flow_file = plan_->getFlowFileProducedByCurrentProcessor();
+        for (const auto& exp_header : expect_header_attributes) {
+          SCOPED_INFO("ConsumeKafka did not produce the expected flowfile attribute from message header: " << exp_header.first << ".");
+          const auto header_attr_opt = flow_file->getAttribute(exp_header.first);
+          REQUIRE(header_attr_opt);
+          REQUIRE(exp_header.second == header_attr_opt.value().get());
+        }
+        {
+          SCOPED_INFO("Message key is missing or incorrect (potential encoding mismatch).");
+          REQUIRE(TEST_MESSAGE_KEY == decode_key(flow_file->getAttribute(ConsumeKafka::KAFKA_MESSAGE_KEY_ATTR).value().get(), key_attribute_encoding));
+          REQUIRE("1" == flow_file->getAttribute(ConsumeKafka::KAFKA_COUNT_ATTR).value().get());
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_OFFSET_ATTR));
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_PARTITION_ATTR));
+          REQUIRE(PRODUCER_TOPIC == flow_file->getAttribute(ConsumeKafka::KAFKA_TOPIC_ATTR).value().get());
+        }
+        flow_files_produced.emplace_back(std::move(flow_file));
+      }
+      plan_->reset_location();
+    }
+
+    const auto contentOrderOfFlowFile = [&] (const std::shared_ptr<core::FlowFile>& lhs, const std::shared_ptr<core::FlowFile>& rhs) {
+      return lhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() < rhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get();
+    };
+    {
+      SCOPED_INFO("The flowfiles generated by ConsumeKafka are invalid (probably nullptr).");
+      REQUIRE_NOTHROW(std::sort(flow_files_produced.begin(), flow_files_produced.end(), contentOrderOfFlowFile));
+    }
+    std::vector<std::string> sorted_split_messages = sort_and_split_messages(messages_on_topic, message_demarcator);
+    const auto flow_file_content_matches_message = [&] (const std::shared_ptr<core::FlowFile>& flowfile, const std::string message) {
+      return flowfile->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() == message;
+    };
+
+    logger_->log_debug("************");
+    std::string expected = "Expected: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      expected += sorted_split_messages[i] + ", ";
+    }
+    std::string   actual = "  Actual: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      actual += flow_files_produced[i]->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() + ", ";
+    }
+    logger_->log_debug("%s", expected.c_str());
+    logger_->log_debug("%s", actual.c_str());
+    logger_->log_debug("************");
+
+    INFO("The messages received by ConsumeKafka do not match those published");
+    REQUIRE(std::equal(flow_files_produced.begin(), flow_files_produced.end(), sorted_split_messages.begin(), flow_file_content_matches_message));
+  }
+};
+
+class ConsumeKafkaContinuousPublishingTest : public ConsumeKafkaTest {
+ public:
+  ConsumeKafkaContinuousPublishingTest() : ConsumeKafkaTest() {}
+  virtual ~ConsumeKafkaContinuousPublishingTest() {
+    logTestController_.reset();
+  }
+
+  void single_consumer_with_continuous_message_producing(
+      const uint64_t msg_periodicity_ms,
+      const std::string& kafka_brokers,
+      const optional<std::string>& group_id,
+      const optional<std::string>& max_poll_records,
+      const optional<std::string>& max_poll_time,
+      const optional<std::string>& session_timeout) {
+    reInitialize();
+
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+
+    plan_->setProperty(consume_kafka, "allow.auto.create.topics", "true", true);  // Seems like the topic tests work without this
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), PRODUCER_TOPIC);
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    consume_kafka->setAutoTerminatedRelationships({success});
+
+    KafkaTestProducer producer("localhost:9092", PRODUCER_TOPIC, /* transactional = */ false);
+
+    std::atomic_bool producer_loop_stop{ false };
+    const auto producer_loop = [&] {
+      std::size_t num_messages_sent = 0;  // When on C++14 make this an initialized lambda capture

Review comment:
       Good catch!




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553402893



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,569 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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.");

Review comment:
       Padding removed.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r538358865



##########
File path: libminifi/test/TestBase.h
##########
@@ -200,7 +199,7 @@ class LogTestController {
     }
     my_properties_->set("logger.root", "ERROR,ostream");
     my_properties_->set("logger." + core::getClassName<LogTestController>(), "INFO");
-    my_properties_->set("logger." + core::getClassName<logging::LoggerConfiguration>(), "DEBUG");
+    my_properties_->set("logger." + core::getClassName<logging::LoggerConfiguration>(), "INFO");

Review comment:
       This breaks C2JstackTest. Once  #953 is merged, rebasing on it should fix this issue. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553906201



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,569 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));
+  }
+  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::steady_clock::now();
+  auto elapsed = std::chrono::steady_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());
+    std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>
+      message { rd_kafka_consumer_poll(consumer_.get(), std::chrono::duration_cast<std::chrono::milliseconds>(max_poll_time_milliseconds_ - elapsed).count()), utils::rd_kafka_message_deleter() };
+    if (!message) {
+      break;
+    }
+    if (RD_KAFKA_RESP_ERR_NO_ERROR != message->err) {
+      logger_->log_error("Received message with error %d: %s", message->err, rd_kafka_err2str(message->err));
+      break;
+    }
+    utils::print_kafka_message(message.get(), logger_);
+    messages.emplace_back(std::move(message));
+    elapsed = std::chrono::steady_clock::now() - start;
+  }
+  return messages;
+}
+
+utils::KafkaEncoding ConsumeKafka::key_attr_encoding_attr_to_enum() const {
+  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() const {
+  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, "\"Message Header Encoding\" property not recognized.");
+}
+
+std::string ConsumeKafka::resolve_duplicate_headers(const std::vector<std::string>& matching_headers) const {
+  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) const {
+  // 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) const {
+  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_) {
+      ret = stream->write(data_,  gsl::narrow<int>(dataSize_));
+    }
+    return ret;
+  }
+};
+
+void ConsumeKafka::add_kafka_attributes_to_flowfile(std::shared_ptr<FlowFileRecord>& flow_file, const rd_kafka_message_t* message) const {
+  // 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, 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));
+}
+
+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) const {
+  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("Message received contains no data.");
+      continue;
+    }
+
+    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 { message_content };
+    if (message_demarcator_.size()) {
+      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) {
+        logger_->log_error("Failed to create flowfile.");
+        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);
+      }
+      add_kafka_attributes_to_flowfile(flow_file, message.get());
+      flow_files_created.emplace_back(std::move(flow_file));
+    }
+  }
+  return flow_files_created;
+}
+
+void ConsumeKafka::onTrigger(core::ProcessContext* /* context */, core::ProcessSession* session) {
+  logger_->log_debug("ConsumeKafka onTrigger");
+
+  std::vector<std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>> messages = poll_kafka_messages();
+
+  std::vector<std::shared_ptr<FlowFileRecord>> flow_files_created = transform_messages_into_flowfiles(messages, session);
+  if (flow_files_created.empty()) {
+    return;
+  }
+  for (const auto& flow_file : flow_files_created) {
+    session->transfer(flow_file, Success);
+  }
+  session->commit();
+  // Commit the offset from the latest message only
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rd_kafka_commit_message(consumer_.get(), messages.back().get(), /* async = */ 0)) {
+    logger_ -> log_error("Committing offsets failed.");

Review comment:
       Seems like this comment is outdated.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553901711



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,569 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));
+  }
+  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::steady_clock::now();
+  auto elapsed = std::chrono::steady_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());
+    std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>
+      message { rd_kafka_consumer_poll(consumer_.get(), std::chrono::duration_cast<std::chrono::milliseconds>(max_poll_time_milliseconds_ - elapsed).count()), utils::rd_kafka_message_deleter() };
+    if (!message) {
+      break;
+    }
+    if (RD_KAFKA_RESP_ERR_NO_ERROR != message->err) {
+      logger_->log_error("Received message with error %d: %s", message->err, rd_kafka_err2str(message->err));
+      break;
+    }
+    utils::print_kafka_message(message.get(), logger_);
+    messages.emplace_back(std::move(message));
+    elapsed = std::chrono::steady_clock::now() - start;
+  }
+  return messages;
+}
+
+utils::KafkaEncoding ConsumeKafka::key_attr_encoding_attr_to_enum() const {
+  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() const {
+  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, "\"Message Header Encoding\" property not recognized.");
+}
+
+std::string ConsumeKafka::resolve_duplicate_headers(const std::vector<std::string>& matching_headers) const {
+  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) const {
+  // 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) const {
+  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_) {
+      ret = stream->write(data_,  gsl::narrow<int>(dataSize_));
+    }
+    return ret;
+  }
+};
+
+void ConsumeKafka::add_kafka_attributes_to_flowfile(std::shared_ptr<FlowFileRecord>& flow_file, const rd_kafka_message_t* message) const {
+  // 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, 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));
+}
+
+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) const {
+  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("Message received contains no data.");
+      continue;
+    }

Review comment:
       Deleted these lines.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553171856



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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);

Review comment:
       I was unaware of the existence of this part of the logger API, thanks. Added `should_log` checks.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553186818



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+      throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker.");

Review comment:
       Added error code and message to both places.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r534926679



##########
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:
       I agree that some of the members are inconsistently named. However, I would rather use snake_case in case of the non-overridden member functions.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] fgerlits commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
fgerlits commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r552445231



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) {

Review comment:
       if we got an error here, should we log it?

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  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();

Review comment:
       This should probably be a `steady_clock`; see the Notes at https://en.cppreference.com/w/cpp/chrono/high_resolution_clock which basically say "don't use this".

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+      throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker.");

Review comment:
       it could be useful to include the error code (or error message, or both) in the text of the exception (also at line 390)

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  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());

Review comment:
       Is this going to leak?  Elsewhere, you wrap the return value into a `unique_ptr`.

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  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() const {
+  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() const {
+  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.");

Review comment:
       typo: should be `Message Header Encoding property`

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  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() const {
+  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() const {
+  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) const {
+  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) const {
+  // 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) const {
+  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;
+  }
+};
+
+void ConsumeKafka::add_kafka_attributes_to_flowfile(std::shared_ptr<FlowFileRecord>& flow_file, const rd_kafka_message_t* message) const {
+  // 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, 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));
+}
+
+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) const {
+  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 {};

Review comment:
       Should this be a `continue` instead of a `return {}`?  If we find an empty message, I'm not sure we want to discard all other messages before and after it.  (Probably at line 515, too.)

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  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() const {
+  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() const {
+  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) const {
+  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) const {
+  // 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()) {

Review comment:
       Should this be an `assert` or a `gsl_Expects` or similar instead of an `if`?  If the length of a header is more than 2 GB, then something must have gone wrong.

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  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() const {
+  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() const {
+  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) const {
+  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) const {
+  // 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) const {
+  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_);

Review comment:
       It's quite unfortunate, but the size parameter of `BaseStream::write()` is an `int`, so this will cause a compiler warning.
   
   At some point, we should probably bite the bullet and rewrite `BaseStream` to use `size_t` or `uint64_t` for stream lengths consistently, but for now, you can either change the type of `dataSize_` to `int` or add a `gsl::narrow` to silence the warning and prevent accidental (though very unlikely) truncation.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553402295



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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() };

Review comment:
       Updated along with the other occurences.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553318298



##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);
+  }
+}
+
+void print_topics_list(std::shared_ptr<logging::Logger> logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger->log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld]",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, const std::shared_ptr<logging::Logger>& logger) {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    const std::string error_msg = "ConsumeKafka: received error message from broker. Librdkafka error msg: " + std::string(rd_kafka_err2str(rkmessage->err));
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, error_msg);
+  }
+  std::string topicName = rd_kafka_topic_name(rkmessage->rkt);
+  std::string message(reinterpret_cast<char*>(rkmessage->payload), rkmessage->len);
+  const char* key = reinterpret_cast<const char*>(rkmessage->key);
+  const std::size_t key_len = rkmessage->key_len;
+  rd_kafka_timestamp_type_t tstype;
+  int64_t timestamp;
+  timestamp = rd_kafka_message_timestamp(rkmessage, &tstype);
+  const char *tsname = "?";
+  if (tstype != RD_KAFKA_TIMESTAMP_NOT_AVAILABLE) {
+    if (tstype == RD_KAFKA_TIMESTAMP_CREATE_TIME) {
+      tsname = "create time";
+    } else if (tstype == RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME) {
+      tsname = "log append time";
+    }
+  }
+  const int64_t seconds_since_timestamp = timestamp ? static_cast<int64_t>(time(NULL)) - static_cast<int64_t>(timestamp / 1000) : 0;
+
+  std::string headers_as_string;
+  rd_kafka_headers_t* hdrs;
+  const rd_kafka_resp_err_t get_header_response = rd_kafka_message_headers(rkmessage, &hdrs);
+  if (RD_KAFKA_RESP_ERR_NO_ERROR == get_header_response) {
+    std::vector<std::string> header_list;
+    kafka_headers_for_each(hdrs, [&] (const std::string& key, const std::string& val) { header_list.emplace_back(key + ": " + val); });
+    headers_as_string = StringUtils::join(", ", header_list);
+  } else if (RD_KAFKA_RESP_ERR__NOENT != 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::string message_as_string;
+  message_as_string += "[Topic](" + topicName + "), ";
+  message_as_string += "[Key](" + (key != nullptr ? std::string(key, key_len) : std::string("[None]")) + "), ";
+  message_as_string += "[Offset](" +  std::to_string(rkmessage->offset) + "), ";
+  message_as_string += "[Message Length](" + std::to_string(rkmessage->len) + "), ";
+  message_as_string += "[Timestamp](" + std::string(tsname) + " " + std::to_string(timestamp) + " (" + std::to_string(seconds_since_timestamp) + " s ago)), ";

Review comment:
       Updated.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r534941010



##########
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:
       Ah, forgotten about this. I would prefer this as the way of including TestBase instead of the long one below. Will correct this by adding the `libminifi/test` include directory to the cmake file. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553173662



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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() };

Review comment:
       Actually, I am not sure it is a safe assumption.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r540107488



##########
File path: libminifi/test/unit/StringUtilsTests.cpp
##########
@@ -50,6 +50,16 @@ TEST_CASE("TestStringUtils::split4", "[test split classname]") {
   REQUIRE(expected == StringUtils::split(org::apache::nifi::minifi::core::getClassName<org::apache::nifi::minifi::utils::StringUtils>(), "::"));
 }
 
+TEST_CASE("TestStringUtils::split5", "[test split delimiter not specified]") {

Review comment:
       There was a `delimiter.at(0)` check in the split implementation.
   
   I will change it to split by characters. However, python and ruby split on whitespace by default:
   https://www.w3schools.com/python/ref_string_split.asp
   https://www.geeksforgeeks.org/ruby-string-split-method-with-examples/




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] fgerlits commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
fgerlits commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r554932057



##########
File path: libminifi/test/TestBase.cpp
##########
@@ -62,78 +63,51 @@ TestPlan::~TestPlan() {
   for (auto& processor : configured_processors_) {
     processor->setScheduledState(core::ScheduledState::STOPPED);
   }
+  for (auto& connection : relationships_) {
+    // This is a patch solving circular references between processors and connections
+    connection->setSource(nullptr);
+    connection->setDestination(nullptr);
+  }
   controller_services_provider_->clearControllerServices();
 }
 
 std::shared_ptr<core::Processor> TestPlan::addProcessor(const std::shared_ptr<core::Processor> &processor, const std::string &name, const std::initializer_list<core::Relationship>& relationships,
-                                                        bool linkToPrevious) {
+    bool linkToPrevious) {
   if (finalized) {
     return nullptr;
   }
   std::lock_guard<std::recursive_mutex> guard(mutex);
-
   utils::Identifier uuid = utils::IdGenerator::getIdGenerator()->generate();
-
   processor->setStreamFactory(stream_factory);
   // initialize the processor
   processor->initialize();
   processor->setFlowIdentifier(flow_version_->getFlowIdentifier());
-
   processor_mapping_[processor->getUUID()] = processor;
-
   if (!linkToPrevious) {
     termination_ = *(relationships.begin());
   } else {
     std::shared_ptr<core::Processor> last = processor_queue_.back();
-
     if (last == nullptr) {
       last = processor;
       termination_ = *(relationships.begin());
     }
-
-    std::stringstream connection_name;
-    connection_name << last->getUUIDStr() << "-to-" << processor->getUUIDStr();
-    logger_->log_info("Creating %s connection for proc %d", connection_name.str(), processor_queue_.size() + 1);
-    std::shared_ptr<minifi::Connection> connection = std::make_shared<minifi::Connection>(flow_repo_, content_repo_, connection_name.str());
-
     for (const auto& relationship : relationships) {
-      connection->addRelationship(relationship);
-    }
-
-    // link the connections so that we can test results at the end for this
-    connection->setSource(last);
-    connection->setDestination(processor);
-
-    connection->setSourceUUID(last->getUUID());
-    connection->setDestinationUUID(processor->getUUID());
-    last->addConnection(connection);
-    if (last != processor) {
-      processor->addConnection(connection);
+      addConnection(last, relationship, processor);

Review comment:
       The behavior has changed between the old code and the new code.  Are you saying the old code was broken, and this fixes the problem?  If so, what was the problem?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r560831601



##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,117 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error: " + error_msg);
+  }
+}
+
+void print_topics_list(logging::Logger& logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger.log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, logging::Logger& logger) {

Review comment:
       I am not exactly sure what you mean. You mean converting the timestamp and headers into a human readable format?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r540164161



##########
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();
+  }
+};

Review comment:
       I think ConsumeKafka.cpp would be fine as it seems to be used in ConsumeKafka only.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r577477152



##########
File path: libminifi/test/TestBase.cpp
##########
@@ -111,28 +109,19 @@ std::shared_ptr<core::Processor> TestPlan::addProcessor(const std::shared_ptr<co
     }
     relationships_.push_back(connection);
   }
-
   std::shared_ptr<core::ProcessorNode> node = std::make_shared<core::ProcessorNode>(processor);
-
   processor_nodes_.push_back(node);
-
   // std::shared_ptr<core::ProcessContext> context = std::make_shared<core::ProcessContext>(node, controller_services_provider_, prov_repo_, flow_repo_, configuration_, content_repo_);
-
   auto contextBuilder = core::ClassLoader::getDefaultClassLoader().instantiate<core::ProcessContextBuilder>("ProcessContextBuilder");
-
   contextBuilder = contextBuilder->withContentRepository(content_repo_)->withFlowFileRepository(flow_repo_)->withProvider(controller_services_provider_.get())->withProvenanceRepository(prov_repo_)->withConfiguration(configuration_);
-
   auto context = contextBuilder->build(node);
-
   processor_contexts_.push_back(context);
-
   processor_queue_.push_back(processor);
-
   return processor;
 }
 
 std::shared_ptr<core::Processor> TestPlan::addProcessor(const std::string &processor_name, const utils::Identifier& uuid, const std::string &name,
-                                                        const std::initializer_list<core::Relationship>& relationships, bool linkToPrevious) {
+  const std::initializer_list<core::Relationship>& relationships, bool linkToPrevious) {

Review comment:
       Updated.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r567675222



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,579 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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 */) {

Review comment:
       Ideally the processor would take references and then this would not be a problem, but since we need to take pointers here (for JNI support IIRC), the next possible safe solution is a wide contract with preconditions.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553336826



##########
File path: extensions/librdkafka/tests/ConsumeKafkaTests.cpp
##########
@@ -0,0 +1,593 @@
+
+/**
+ * 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 "../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(), "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) {
+    reInitialize();
+
+    // Consumer chain
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+    std::shared_ptr<core::Processor> extract_text  = plan_->addProcessor("ExtractText", "extract_text", {success}, false);
+
+    // Set up connections
+    plan_->addConnection(consume_kafka, success, extract_text);
+    extract_text->setAutoTerminatedRelationships({success});
+
+    const auto bool_to_string = [] (const bool b) -> std::string { return b ? "true" : "false"; };
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::SecurityProtocol.getName(), security_protocol);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), topic_names);
+
+    optional_set_property(consume_kafka, ConsumeKafka::TopicNameFormat.getName(), topic_name_format);
+    optional_set_property(consume_kafka, ConsumeKafka::HonorTransactions.getName(), honor_transactions | utils::map(bool_to_string));
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+    optional_set_property(consume_kafka, ConsumeKafka::OffsetReset.getName(), offset_reset);
+    optional_set_property(consume_kafka, ConsumeKafka::KeyAttributeEncoding.getName(), key_attribute_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageDemarcator.getName(), message_demarcator);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageHeaderEncoding.getName(), message_header_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::HeadersToAddAsAttributes.getName(), headers_to_add_as_attributes);
+    optional_set_property(consume_kafka, ConsumeKafka::DuplicateHeaderHandling.getName(), duplicate_header_handling);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    plan_->setProperty(extract_text, ExtractText::Attribute.getName(), ATTRIBUTE_FOR_CAPTURING_CONTENT);
+
+    if (!expect_config_valid) {
+      const auto& message = messages_on_topic.front();
+      REQUIRE_THROWS(plan_->scheduleProcessor(consume_kafka));
+      return;
+    } else {
+      plan_->scheduleProcessors();
+    }
+
+    std::unique_ptr<rd_kafka_conf_t, utils::rd_kafka_conf_deleter> conf_;
+    std::unique_ptr<rd_kafka_t, utils::rd_kafka_consumer_deleter> consumer_;
+
+    const bool is_transactional = std::count(transaction_events.cbegin(), transaction_events.cend(), KafkaTestProducer::PublishEvent::TRANSACTION_START);
+    const bool transactions_committed = transaction_events.back() == KafkaTestProducer::PublishEvent::TRANSACTION_COMMIT;
+
+    KafkaTestProducer producer(kafka_brokers, PRODUCER_TOPIC, is_transactional);
+    producer.publish_messages_to_topic(messages_on_topic, TEST_MESSAGE_KEY, transaction_events, message_headers, message_header_encoding);
+
+
+    std::vector<std::shared_ptr<core::FlowFile>> flow_files_produced;
+    for (std::size_t num_expected_messages_processed = 0; num_expected_messages_processed < messages_on_topic.size(); num_expected_messages_processed += std::stoi(max_poll_records.value_or("1"))) {
+      plan_->increment_location();
+      if ((honor_transactions && false == honor_transactions.value()) || (is_transactional && !transactions_committed)) {
+        INFO("Non-committed messages received.");
+        REQUIRE(false == plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+        return;
+      }
+      {
+        SCOPED_INFO("ConsumeKafka timed out when waiting to receive the message published to the kafka broker.");
+        REQUIRE(plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+      }
+      std::size_t num_flow_files_produced = plan_->getNumFlowFileProducedByCurrentProcessor();
+      plan_->increment_location();
+      for (std::size_t times_extract_text_run = 0; times_extract_text_run < num_flow_files_produced; ++times_extract_text_run) {
+        plan_->runCurrentProcessor();  // ExtractText
+        std::shared_ptr<core::FlowFile> flow_file = plan_->getFlowFileProducedByCurrentProcessor();
+        for (const auto& exp_header : expect_header_attributes) {
+          SCOPED_INFO("ConsumeKafka did not produce the expected flowfile attribute from message header: " << exp_header.first << ".");
+          const auto header_attr_opt = flow_file->getAttribute(exp_header.first);
+          REQUIRE(header_attr_opt);
+          REQUIRE(exp_header.second == header_attr_opt.value().get());
+        }
+        {
+          SCOPED_INFO("Message key is missing or incorrect (potential encoding mismatch).");
+          REQUIRE(TEST_MESSAGE_KEY == decode_key(flow_file->getAttribute(ConsumeKafka::KAFKA_MESSAGE_KEY_ATTR).value().get(), key_attribute_encoding));
+          REQUIRE("1" == flow_file->getAttribute(ConsumeKafka::KAFKA_COUNT_ATTR).value().get());
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_OFFSET_ATTR));
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_PARTITION_ATTR));
+          REQUIRE(PRODUCER_TOPIC == flow_file->getAttribute(ConsumeKafka::KAFKA_TOPIC_ATTR).value().get());
+        }
+        flow_files_produced.emplace_back(std::move(flow_file));
+      }
+      plan_->reset_location();
+    }
+
+    const auto contentOrderOfFlowFile = [&] (const std::shared_ptr<core::FlowFile>& lhs, const std::shared_ptr<core::FlowFile>& rhs) {
+      return lhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() < rhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get();
+    };
+    {
+      SCOPED_INFO("The flowfiles generated by ConsumeKafka are invalid (probably nullptr).");
+      CHECK_NOTHROW(std::sort(flow_files_produced.begin(), flow_files_produced.end(), contentOrderOfFlowFile));
+    }
+    std::vector<std::string> sorted_split_messages = sort_and_split_messages(messages_on_topic, message_demarcator);
+    const auto flow_file_content_matches_message = [&] (const std::shared_ptr<core::FlowFile>& flowfile, const std::string message) {
+      return flowfile->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() == message;
+    };
+
+    logger_->log_debug("************");
+    std::string expected = "Expected: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      expected += sorted_split_messages[i] + ", ";
+    }
+    std::string   actual = "  Actual: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      actual += flow_files_produced[i]->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() + ", ";
+    }
+    logger_->log_debug("%s", expected.c_str());
+    logger_->log_debug("%s", actual.c_str());
+    logger_->log_debug("************");
+
+    INFO("The messages received by ConsumeKafka do not match those published");
+    REQUIRE(std::equal(flow_files_produced.begin(), flow_files_produced.end(), sorted_split_messages.begin(), flow_file_content_matches_message));
+  }
+};
+
+class ConsumeKafkaContinuousPublishingTest : public ConsumeKafkaTest {
+ public:
+  ConsumeKafkaContinuousPublishingTest() : ConsumeKafkaTest() {}
+  virtual ~ConsumeKafkaContinuousPublishingTest() {
+    logTestController_.reset();
+  }
+
+  void single_consumer_with_continuous_message_producing(
+      const uint64_t msg_periodicity_ms,
+      const std::string& kafka_brokers,
+      const optional<std::string>& group_id,
+      const optional<std::string>& max_poll_records,
+      const optional<std::string>& max_poll_time,
+      const optional<std::string>& session_timeout) {
+    reInitialize();
+
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+
+    plan_->setProperty(consume_kafka, "allow.auto.create.topics", "true", true);  // Seems like the topic tests work without this
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), PRODUCER_TOPIC);
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    consume_kafka->setAutoTerminatedRelationships({success});
+
+    KafkaTestProducer producer("localhost:9092", PRODUCER_TOPIC, /* transactional = */ false);
+
+    std::atomic_bool producer_loop_stop{ false };
+    std::size_t num_messages_sent = 0;  // When on C++14 make this an initialized lambda capture

Review comment:
       I wonder what I was going for here. I think it can even be removed from being a value returned.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553906593



##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);
+  }
+}
+
+void print_topics_list(std::shared_ptr<logging::Logger> logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger->log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld]",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, const std::shared_ptr<logging::Logger>& logger) {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    const std::string error_msg = "ConsumeKafka: received error message from broker. Librdkafka error msg: " + std::string(rd_kafka_err2str(rkmessage->err));
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, error_msg);
+  }
+  std::string topicName = rd_kafka_topic_name(rkmessage->rkt);
+  std::string message(reinterpret_cast<char*>(rkmessage->payload), rkmessage->len);
+  const char* key = reinterpret_cast<const char*>(rkmessage->key);
+  const std::size_t key_len = rkmessage->key_len;
+  rd_kafka_timestamp_type_t tstype;
+  int64_t timestamp;
+  timestamp = rd_kafka_message_timestamp(rkmessage, &tstype);
+  const char *tsname = "?";
+  if (tstype != RD_KAFKA_TIMESTAMP_NOT_AVAILABLE) {
+    if (tstype == RD_KAFKA_TIMESTAMP_CREATE_TIME) {
+      tsname = "create time";
+    } else if (tstype == RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME) {
+      tsname = "log append time";
+    }
+  }
+  const int64_t seconds_since_timestamp = timestamp ? static_cast<int64_t>(time(NULL)) - static_cast<int64_t>(timestamp / 1000) : 0;

Review comment:
       It does print the absolute one as well in this format:
   > [Timestamp](create time 1610107476940 (0 s ago)),




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553202896



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  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() const {
+  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() const {
+  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) const {
+  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) const {
+  // 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) const {
+  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_);

Review comment:
       Yep, during the implementation of ConsumeKafka, the api of the stream writer suddenly changed and I thought just matching the types is good enough. It really makes no sense to check the sign of an `uint64_t`. Updated to use `gsl::narrow`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] fgerlits commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
fgerlits commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553417399



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  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() const {
+  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() const {
+  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) const {
+  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) const {
+  // 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()) {

Review comment:
       These are headers, not messages.  Is a single header larger than 2 GB a valid use case?
   
   Also, logging a 2 GB long string is probably not a good idea.  It would probably be better to log the first 100 or 200 characters, followed by "..." if there are more.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553893682



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,569 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));

Review comment:
       I don't mind concatenation when building an error.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553396875



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,569 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {

Review comment:
       Removed.

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,569 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {

Review comment:
       Removed.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553850276



##########
File path: libminifi/include/utils/ProcessorConfigUtils.h
##########
@@ -0,0 +1,80 @@
+/**
+ * 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) {
+  std::string value;
+  if (!context->getProperty(property_name, value)) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property missing or invalid");
+  }
+  return value;
+}
+
+std::vector<std::string> listFromCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  std::string property_string;
+  context->getProperty(property_name, property_string);
+  return utils::StringUtils::splitAndTrim(property_string, ",");
+}
+
+std::vector<std::string> listFromRequiredCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  return utils::StringUtils::splitAndTrim(getRequiredPropertyOrThrow(context, property_name), ",");
+}
+
+bool parseBooleanPropertyOrThrow(core::ProcessContext* context, const std::string& property_name) {
+  bool value;
+  std::string value_str = getRequiredPropertyOrThrow(context, property_name);
+  utils::optional<bool> maybe_value = utils::StringUtils::toBool(value_str);
+  if (!maybe_value) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property is invalid: value is " + value_str);

Review comment:
       I'm fine with `std::runtime_error` or `std::invalid_argument` as well, but I don't want miscategorized exceptions.

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);
+  }
+}
+
+void print_topics_list(std::shared_ptr<logging::Logger> logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger->log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld]",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, const std::shared_ptr<logging::Logger>& logger) {

Review comment:
       Taking a shared pointer instead of an observer pointer/reference makes it impossible to use the function with anything other than a shared pointer, like unique ptr, stack object, manually allocated or member of another object.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r534272416



##########
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:
       Good catch. Also the check should be case-insensitive. Will correct this.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553176879



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) {

Review comment:
       It is a strange case. We only fetch these messages because the offset reset of the kafka API does not seem to always properly align with the latest offset. As we are discarding these messages either way, I think logging their validity might be misleading.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] fgerlits commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
fgerlits commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553412286



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,582 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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());

Review comment:
       Please update PROCESSORS.md, as well. (Required properties should be in bold.)

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  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() const {
+  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() const {
+  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) const {
+  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) const {
+  // 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()) {

Review comment:
       These are headers, not messages.  Is a single header larger than 2 GB a valid use case?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r534998855



##########
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:
       Thanks!




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
lordgamez commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r534990092



##########
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:
       I see I suppose you are right in case it is explicit what parameters are used as they are fixed and it can be set in a single line. Let's keep it as it is, I'm not against it. The only downside I see is that because of the long list of string parameters it could be easy to mix them up when calling, but I suppose it can be easily realized after the test fails.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553313197



##########
File path: extensions/librdkafka/ConsumeKafka.h
##########
@@ -0,0 +1,175 @@
+/**
+ * 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 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 create_topic_partition_list();
+  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) const;
+  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() const;
+  utils::KafkaEncoding message_header_encoding_attr_to_enum() const;
+  std::string resolve_duplicate_headers(const std::vector<std::string>& matching_headers) const;
+  std::vector<std::string> get_matching_headers(const rd_kafka_message_t* message, const std::string& header_name) const;
+  std::vector<std::pair<std::string, std::string>> get_flowfile_attributes_from_message_header(const rd_kafka_message_t* message) const;
+  void add_kafka_attributes_to_flowfile(std::shared_ptr<FlowFileRecord>& flow_file, const rd_kafka_message_t* message) const;
+  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) const;
+
+ private:
+  std::string kafka_brokers_;
+  std::string security_protocol_;
+  std::vector<std::string> topic_names_;
+  std::string topic_name_format_;
+  bool honor_transactions_;
+  std::string group_id_;
+  std::string offset_reset_;
+  std::string key_attribute_encoding_;
+  std::string message_demarcator_;
+  std::string message_header_encoding_;
+  std::string duplicate_header_handling_;
+  std::vector<std::string> headers_to_add_as_attributes_;
+  std::size_t max_poll_records_;
+  std::chrono::milliseconds max_poll_time_milliseconds_;
+  std::chrono::milliseconds communications_timeout_milliseconds_;
+  std::chrono::milliseconds session_timeout_milliseconds_;

Review comment:
       I was thinking where to store this info to make it obvious. Having an explicit name is more informative when reading the code, but having the type makes it safer.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r534973423



##########
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:
       Will change this to camelCase.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r560990362



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,579 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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.")

Review comment:
       How are messages bundled together into a flow file? I thought we emit one flow file per message.

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,579 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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 */) {

Review comment:
       I would consider checking that `context` is valid and throwing if not. Alternatively, setting a precondition with `gsl_Expects(context)` is fine too.

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,579 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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) {

Review comment:
       
   
   This could take a reference instead of a pointer. Currently its signature allows for null but it crashes on null.
   

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,579 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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:
       This could take a reference instead of a pointer. Currently its signature allows for null but it crashes on null.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r576730170



##########
File path: libminifi/test/TestBase.cpp
##########
@@ -111,28 +109,19 @@ std::shared_ptr<core::Processor> TestPlan::addProcessor(const std::shared_ptr<co
     }
     relationships_.push_back(connection);
   }
-
   std::shared_ptr<core::ProcessorNode> node = std::make_shared<core::ProcessorNode>(processor);
-
   processor_nodes_.push_back(node);
-
   // std::shared_ptr<core::ProcessContext> context = std::make_shared<core::ProcessContext>(node, controller_services_provider_, prov_repo_, flow_repo_, configuration_, content_repo_);
-
   auto contextBuilder = core::ClassLoader::getDefaultClassLoader().instantiate<core::ProcessContextBuilder>("ProcessContextBuilder");
-
   contextBuilder = contextBuilder->withContentRepository(content_repo_)->withFlowFileRepository(flow_repo_)->withProvider(controller_services_provider_.get())->withProvenanceRepository(prov_repo_)->withConfiguration(configuration_);
-
   auto context = contextBuilder->build(node);
-
   processor_contexts_.push_back(context);
-
   processor_queue_.push_back(processor);
-
   return processor;
 }
 
 std::shared_ptr<core::Processor> TestPlan::addProcessor(const std::string &processor_name, const utils::Identifier& uuid, const std::string &name,
-                                                        const std::initializer_list<core::Relationship>& relationships, bool linkToPrevious) {
+  const std::initializer_list<core::Relationship>& relationships, bool linkToPrevious) {

Review comment:
       Continuation indentation should be 4 spaces or aligned. My preference is 4 spaces (or more generally 2 levels of indentation), the style guide's is aligned.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553322678



##########
File path: extensions/librdkafka/tests/ConsumeKafkaTests.cpp
##########
@@ -0,0 +1,593 @@
+
+/**
+ * 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 "../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(), "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 };

Review comment:
       Updated as suggested.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553339414



##########
File path: extensions/librdkafka/tests/ConsumeKafkaTests.cpp
##########
@@ -0,0 +1,593 @@
+
+/**
+ * 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 "../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(), "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) {
+    reInitialize();
+
+    // Consumer chain
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+    std::shared_ptr<core::Processor> extract_text  = plan_->addProcessor("ExtractText", "extract_text", {success}, false);
+
+    // Set up connections
+    plan_->addConnection(consume_kafka, success, extract_text);
+    extract_text->setAutoTerminatedRelationships({success});
+
+    const auto bool_to_string = [] (const bool b) -> std::string { return b ? "true" : "false"; };
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::SecurityProtocol.getName(), security_protocol);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), topic_names);
+
+    optional_set_property(consume_kafka, ConsumeKafka::TopicNameFormat.getName(), topic_name_format);
+    optional_set_property(consume_kafka, ConsumeKafka::HonorTransactions.getName(), honor_transactions | utils::map(bool_to_string));
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+    optional_set_property(consume_kafka, ConsumeKafka::OffsetReset.getName(), offset_reset);
+    optional_set_property(consume_kafka, ConsumeKafka::KeyAttributeEncoding.getName(), key_attribute_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageDemarcator.getName(), message_demarcator);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageHeaderEncoding.getName(), message_header_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::HeadersToAddAsAttributes.getName(), headers_to_add_as_attributes);
+    optional_set_property(consume_kafka, ConsumeKafka::DuplicateHeaderHandling.getName(), duplicate_header_handling);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    plan_->setProperty(extract_text, ExtractText::Attribute.getName(), ATTRIBUTE_FOR_CAPTURING_CONTENT);
+
+    if (!expect_config_valid) {
+      const auto& message = messages_on_topic.front();
+      REQUIRE_THROWS(plan_->scheduleProcessor(consume_kafka));
+      return;
+    } else {
+      plan_->scheduleProcessors();
+    }
+
+    std::unique_ptr<rd_kafka_conf_t, utils::rd_kafka_conf_deleter> conf_;
+    std::unique_ptr<rd_kafka_t, utils::rd_kafka_consumer_deleter> consumer_;
+
+    const bool is_transactional = std::count(transaction_events.cbegin(), transaction_events.cend(), KafkaTestProducer::PublishEvent::TRANSACTION_START);
+    const bool transactions_committed = transaction_events.back() == KafkaTestProducer::PublishEvent::TRANSACTION_COMMIT;
+
+    KafkaTestProducer producer(kafka_brokers, PRODUCER_TOPIC, is_transactional);
+    producer.publish_messages_to_topic(messages_on_topic, TEST_MESSAGE_KEY, transaction_events, message_headers, message_header_encoding);
+
+
+    std::vector<std::shared_ptr<core::FlowFile>> flow_files_produced;
+    for (std::size_t num_expected_messages_processed = 0; num_expected_messages_processed < messages_on_topic.size(); num_expected_messages_processed += std::stoi(max_poll_records.value_or("1"))) {
+      plan_->increment_location();
+      if ((honor_transactions && false == honor_transactions.value()) || (is_transactional && !transactions_committed)) {
+        INFO("Non-committed messages received.");
+        REQUIRE(false == plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+        return;
+      }
+      {
+        SCOPED_INFO("ConsumeKafka timed out when waiting to receive the message published to the kafka broker.");
+        REQUIRE(plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+      }
+      std::size_t num_flow_files_produced = plan_->getNumFlowFileProducedByCurrentProcessor();
+      plan_->increment_location();
+      for (std::size_t times_extract_text_run = 0; times_extract_text_run < num_flow_files_produced; ++times_extract_text_run) {
+        plan_->runCurrentProcessor();  // ExtractText
+        std::shared_ptr<core::FlowFile> flow_file = plan_->getFlowFileProducedByCurrentProcessor();
+        for (const auto& exp_header : expect_header_attributes) {
+          SCOPED_INFO("ConsumeKafka did not produce the expected flowfile attribute from message header: " << exp_header.first << ".");
+          const auto header_attr_opt = flow_file->getAttribute(exp_header.first);
+          REQUIRE(header_attr_opt);
+          REQUIRE(exp_header.second == header_attr_opt.value().get());
+        }
+        {
+          SCOPED_INFO("Message key is missing or incorrect (potential encoding mismatch).");
+          REQUIRE(TEST_MESSAGE_KEY == decode_key(flow_file->getAttribute(ConsumeKafka::KAFKA_MESSAGE_KEY_ATTR).value().get(), key_attribute_encoding));
+          REQUIRE("1" == flow_file->getAttribute(ConsumeKafka::KAFKA_COUNT_ATTR).value().get());
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_OFFSET_ATTR));
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_PARTITION_ATTR));
+          REQUIRE(PRODUCER_TOPIC == flow_file->getAttribute(ConsumeKafka::KAFKA_TOPIC_ATTR).value().get());
+        }
+        flow_files_produced.emplace_back(std::move(flow_file));
+      }
+      plan_->reset_location();
+    }
+
+    const auto contentOrderOfFlowFile = [&] (const std::shared_ptr<core::FlowFile>& lhs, const std::shared_ptr<core::FlowFile>& rhs) {
+      return lhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() < rhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get();
+    };
+    {
+      SCOPED_INFO("The flowfiles generated by ConsumeKafka are invalid (probably nullptr).");
+      CHECK_NOTHROW(std::sort(flow_files_produced.begin(), flow_files_produced.end(), contentOrderOfFlowFile));
+    }
+    std::vector<std::string> sorted_split_messages = sort_and_split_messages(messages_on_topic, message_demarcator);
+    const auto flow_file_content_matches_message = [&] (const std::shared_ptr<core::FlowFile>& flowfile, const std::string message) {
+      return flowfile->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() == message;
+    };
+
+    logger_->log_debug("************");
+    std::string expected = "Expected: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      expected += sorted_split_messages[i] + ", ";
+    }
+    std::string   actual = "  Actual: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      actual += flow_files_produced[i]->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() + ", ";
+    }
+    logger_->log_debug("%s", expected.c_str());
+    logger_->log_debug("%s", actual.c_str());
+    logger_->log_debug("************");
+
+    INFO("The messages received by ConsumeKafka do not match those published");
+    REQUIRE(std::equal(flow_files_produced.begin(), flow_files_produced.end(), sorted_split_messages.begin(), flow_file_content_matches_message));
+  }
+};
+
+class ConsumeKafkaContinuousPublishingTest : public ConsumeKafkaTest {
+ public:
+  ConsumeKafkaContinuousPublishingTest() : ConsumeKafkaTest() {}
+  virtual ~ConsumeKafkaContinuousPublishingTest() {
+    logTestController_.reset();
+  }
+
+  void single_consumer_with_continuous_message_producing(
+      const uint64_t msg_periodicity_ms,
+      const std::string& kafka_brokers,
+      const optional<std::string>& group_id,
+      const optional<std::string>& max_poll_records,
+      const optional<std::string>& max_poll_time,
+      const optional<std::string>& session_timeout) {
+    reInitialize();
+
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+
+    plan_->setProperty(consume_kafka, "allow.auto.create.topics", "true", true);  // Seems like the topic tests work without this
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), PRODUCER_TOPIC);
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    consume_kafka->setAutoTerminatedRelationships({success});
+
+    KafkaTestProducer producer("localhost:9092", PRODUCER_TOPIC, /* transactional = */ false);
+
+    std::atomic_bool producer_loop_stop{ false };
+    std::size_t num_messages_sent = 0;  // When on C++14 make this an initialized lambda capture
+    const auto producer_loop = [&] {
+      std::this_thread::sleep_for(std::chrono::milliseconds(100));
+      while (!producer_loop_stop) {
+        producer.publish_messages_to_topic({ "Message after " + std::to_string(msg_periodicity_ms * num_messages_sent) + " ms"}, TEST_MESSAGE_KEY, { PUBLISH }, {}, {});
+        ++num_messages_sent;
+        std::this_thread::sleep_for(std::chrono::milliseconds(msg_periodicity_ms));
+      }
+      return num_messages_sent;
+    };
+
+    plan_->scheduleProcessors();
+
+    const auto get_time_property_ms = [] (const std::string& property_string) {
+      int64_t value;
+      org::apache::nifi::minifi::core::TimeUnit unit;
+      REQUIRE(org::apache::nifi::minifi::core::Property::StringToTime(property_string, value, unit));
+      int64_t value_as_ms;
+      REQUIRE(org::apache::nifi::minifi::core::Property::ConvertTimeUnitToMS(value, unit, value_as_ms));
+      return value_as_ms;
+    };
+
+    std::thread producer_thread(producer_loop);
+    CHECK_NOTHROW(plan_->runNextProcessor());
+    producer_loop_stop = true;
+    producer_thread.join();

Review comment:
       Talked through privately, this was accepted to continue running in order to provide data on how many flowfiles were actually produced.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r551922933



##########
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) {
+  std::string value;
+  if (!context->getProperty(property_name, value)) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property missing or invalid");
+  }
+  return value;
+}
+
+std::vector<std::string> listFromCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  std::string property_string;
+  context->getProperty(property_name, property_string);
+  return utils::StringUtils::splitAndTrim(property_string, ",");
+}
+
+std::vector<std::string> listFromRequiredCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  return utils::StringUtils::splitAndTrim(getRequiredPropertyOrThrow(context, property_name), ",");
+}
+
+bool parseBooleanPropertyOrThrow(core::ProcessContext* context, const std::string& property_name) {
+  bool value;
+  std::string value_str = getRequiredPropertyOrThrow(context, property_name);
+  return utils::StringUtils::StringToBool(value_str, value);

Review comment:
       Replaced as requested.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r552525343



##########
File path: extensions/librdkafka/ConsumeKafka.h
##########
@@ -0,0 +1,175 @@
+/**
+ * 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 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 create_topic_partition_list();
+  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) const;
+  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() const;
+  utils::KafkaEncoding message_header_encoding_attr_to_enum() const;
+  std::string resolve_duplicate_headers(const std::vector<std::string>& matching_headers) const;
+  std::vector<std::string> get_matching_headers(const rd_kafka_message_t* message, const std::string& header_name) const;
+  std::vector<std::pair<std::string, std::string>> get_flowfile_attributes_from_message_header(const rd_kafka_message_t* message) const;
+  void add_kafka_attributes_to_flowfile(std::shared_ptr<FlowFileRecord>& flow_file, const rd_kafka_message_t* message) const;
+  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) const;
+
+ private:
+  std::string kafka_brokers_;
+  std::string security_protocol_;
+  std::vector<std::string> topic_names_;
+  std::string topic_name_format_;
+  bool honor_transactions_;
+  std::string group_id_;
+  std::string offset_reset_;
+  std::string key_attribute_encoding_;
+  std::string message_demarcator_;
+  std::string message_header_encoding_;
+  std::string duplicate_header_handling_;
+  std::vector<std::string> headers_to_add_as_attributes_;
+  std::size_t max_poll_records_;
+  std::chrono::milliseconds max_poll_time_milliseconds_;
+  std::chrono::milliseconds communications_timeout_milliseconds_;
+  std::chrono::milliseconds session_timeout_milliseconds_;

Review comment:
       I think the `_milliseconds` suffixes in the identifiers are unnecessary because the information is already part of the type.

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);

Review comment:
       The librdkafka api docs don't mention regex matching. Where is this documented?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553336826



##########
File path: extensions/librdkafka/tests/ConsumeKafkaTests.cpp
##########
@@ -0,0 +1,593 @@
+
+/**
+ * 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 "../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(), "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) {
+    reInitialize();
+
+    // Consumer chain
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+    std::shared_ptr<core::Processor> extract_text  = plan_->addProcessor("ExtractText", "extract_text", {success}, false);
+
+    // Set up connections
+    plan_->addConnection(consume_kafka, success, extract_text);
+    extract_text->setAutoTerminatedRelationships({success});
+
+    const auto bool_to_string = [] (const bool b) -> std::string { return b ? "true" : "false"; };
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::SecurityProtocol.getName(), security_protocol);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), topic_names);
+
+    optional_set_property(consume_kafka, ConsumeKafka::TopicNameFormat.getName(), topic_name_format);
+    optional_set_property(consume_kafka, ConsumeKafka::HonorTransactions.getName(), honor_transactions | utils::map(bool_to_string));
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+    optional_set_property(consume_kafka, ConsumeKafka::OffsetReset.getName(), offset_reset);
+    optional_set_property(consume_kafka, ConsumeKafka::KeyAttributeEncoding.getName(), key_attribute_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageDemarcator.getName(), message_demarcator);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageHeaderEncoding.getName(), message_header_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::HeadersToAddAsAttributes.getName(), headers_to_add_as_attributes);
+    optional_set_property(consume_kafka, ConsumeKafka::DuplicateHeaderHandling.getName(), duplicate_header_handling);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    plan_->setProperty(extract_text, ExtractText::Attribute.getName(), ATTRIBUTE_FOR_CAPTURING_CONTENT);
+
+    if (!expect_config_valid) {
+      const auto& message = messages_on_topic.front();
+      REQUIRE_THROWS(plan_->scheduleProcessor(consume_kafka));
+      return;
+    } else {
+      plan_->scheduleProcessors();
+    }
+
+    std::unique_ptr<rd_kafka_conf_t, utils::rd_kafka_conf_deleter> conf_;
+    std::unique_ptr<rd_kafka_t, utils::rd_kafka_consumer_deleter> consumer_;
+
+    const bool is_transactional = std::count(transaction_events.cbegin(), transaction_events.cend(), KafkaTestProducer::PublishEvent::TRANSACTION_START);
+    const bool transactions_committed = transaction_events.back() == KafkaTestProducer::PublishEvent::TRANSACTION_COMMIT;
+
+    KafkaTestProducer producer(kafka_brokers, PRODUCER_TOPIC, is_transactional);
+    producer.publish_messages_to_topic(messages_on_topic, TEST_MESSAGE_KEY, transaction_events, message_headers, message_header_encoding);
+
+
+    std::vector<std::shared_ptr<core::FlowFile>> flow_files_produced;
+    for (std::size_t num_expected_messages_processed = 0; num_expected_messages_processed < messages_on_topic.size(); num_expected_messages_processed += std::stoi(max_poll_records.value_or("1"))) {
+      plan_->increment_location();
+      if ((honor_transactions && false == honor_transactions.value()) || (is_transactional && !transactions_committed)) {
+        INFO("Non-committed messages received.");
+        REQUIRE(false == plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+        return;
+      }
+      {
+        SCOPED_INFO("ConsumeKafka timed out when waiting to receive the message published to the kafka broker.");
+        REQUIRE(plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+      }
+      std::size_t num_flow_files_produced = plan_->getNumFlowFileProducedByCurrentProcessor();
+      plan_->increment_location();
+      for (std::size_t times_extract_text_run = 0; times_extract_text_run < num_flow_files_produced; ++times_extract_text_run) {
+        plan_->runCurrentProcessor();  // ExtractText
+        std::shared_ptr<core::FlowFile> flow_file = plan_->getFlowFileProducedByCurrentProcessor();
+        for (const auto& exp_header : expect_header_attributes) {
+          SCOPED_INFO("ConsumeKafka did not produce the expected flowfile attribute from message header: " << exp_header.first << ".");
+          const auto header_attr_opt = flow_file->getAttribute(exp_header.first);
+          REQUIRE(header_attr_opt);
+          REQUIRE(exp_header.second == header_attr_opt.value().get());
+        }
+        {
+          SCOPED_INFO("Message key is missing or incorrect (potential encoding mismatch).");
+          REQUIRE(TEST_MESSAGE_KEY == decode_key(flow_file->getAttribute(ConsumeKafka::KAFKA_MESSAGE_KEY_ATTR).value().get(), key_attribute_encoding));
+          REQUIRE("1" == flow_file->getAttribute(ConsumeKafka::KAFKA_COUNT_ATTR).value().get());
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_OFFSET_ATTR));
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_PARTITION_ATTR));
+          REQUIRE(PRODUCER_TOPIC == flow_file->getAttribute(ConsumeKafka::KAFKA_TOPIC_ATTR).value().get());
+        }
+        flow_files_produced.emplace_back(std::move(flow_file));
+      }
+      plan_->reset_location();
+    }
+
+    const auto contentOrderOfFlowFile = [&] (const std::shared_ptr<core::FlowFile>& lhs, const std::shared_ptr<core::FlowFile>& rhs) {
+      return lhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() < rhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get();
+    };
+    {
+      SCOPED_INFO("The flowfiles generated by ConsumeKafka are invalid (probably nullptr).");
+      CHECK_NOTHROW(std::sort(flow_files_produced.begin(), flow_files_produced.end(), contentOrderOfFlowFile));
+    }
+    std::vector<std::string> sorted_split_messages = sort_and_split_messages(messages_on_topic, message_demarcator);
+    const auto flow_file_content_matches_message = [&] (const std::shared_ptr<core::FlowFile>& flowfile, const std::string message) {
+      return flowfile->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() == message;
+    };
+
+    logger_->log_debug("************");
+    std::string expected = "Expected: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      expected += sorted_split_messages[i] + ", ";
+    }
+    std::string   actual = "  Actual: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      actual += flow_files_produced[i]->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() + ", ";
+    }
+    logger_->log_debug("%s", expected.c_str());
+    logger_->log_debug("%s", actual.c_str());
+    logger_->log_debug("************");
+
+    INFO("The messages received by ConsumeKafka do not match those published");
+    REQUIRE(std::equal(flow_files_produced.begin(), flow_files_produced.end(), sorted_split_messages.begin(), flow_file_content_matches_message));
+  }
+};
+
+class ConsumeKafkaContinuousPublishingTest : public ConsumeKafkaTest {
+ public:
+  ConsumeKafkaContinuousPublishingTest() : ConsumeKafkaTest() {}
+  virtual ~ConsumeKafkaContinuousPublishingTest() {
+    logTestController_.reset();
+  }
+
+  void single_consumer_with_continuous_message_producing(
+      const uint64_t msg_periodicity_ms,
+      const std::string& kafka_brokers,
+      const optional<std::string>& group_id,
+      const optional<std::string>& max_poll_records,
+      const optional<std::string>& max_poll_time,
+      const optional<std::string>& session_timeout) {
+    reInitialize();
+
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+
+    plan_->setProperty(consume_kafka, "allow.auto.create.topics", "true", true);  // Seems like the topic tests work without this
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), PRODUCER_TOPIC);
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    consume_kafka->setAutoTerminatedRelationships({success});
+
+    KafkaTestProducer producer("localhost:9092", PRODUCER_TOPIC, /* transactional = */ false);
+
+    std::atomic_bool producer_loop_stop{ false };
+    std::size_t num_messages_sent = 0;  // When on C++14 make this an initialized lambda capture

Review comment:
       I wonder what I was going for here. I think it can even be deleted now.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553315141



##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };

Review comment:
       Same as above, actually, I am not sure if it is a safe assumption.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553477912



##########
File path: libminifi/src/utils/StringUtils.cpp
##########
@@ -46,8 +46,13 @@ std::string StringUtils::trim(const std::string& s) {
   return trimRight(trimLeft(s));
 }
 
-std::vector<std::string> StringUtils::split(const std::string &str, const std::string &delimiter) {
+template<typename Fun>
+std::vector<std::string> split_transformed(const std::string& str, const std::string& delimiter, Fun transformation) {
   std::vector<std::string> result;
+  if (delimiter.empty()) {
+    std::transform(str.begin(), str.end(), std::back_inserter(result), [] (const char c) { return std::string{c}; });

Review comment:
       This branch should apply `transformation` as well.

##########
File path: libminifi/include/utils/ProcessorConfigUtils.h
##########
@@ -0,0 +1,80 @@
+/**
+ * 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) {
+  std::string value;
+  if (!context->getProperty(property_name, value)) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property missing or invalid");
+  }
+  return value;
+}
+
+std::vector<std::string> listFromCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  std::string property_string;
+  context->getProperty(property_name, property_string);
+  return utils::StringUtils::splitAndTrim(property_string, ",");
+}
+
+std::vector<std::string> listFromRequiredCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  return utils::StringUtils::splitAndTrim(getRequiredPropertyOrThrow(context, property_name), ",");
+}
+
+bool parseBooleanPropertyOrThrow(core::ProcessContext* context, const std::string& property_name) {
+  bool value;
+  std::string value_str = getRequiredPropertyOrThrow(context, property_name);
+  utils::optional<bool> maybe_value = utils::StringUtils::toBool(value_str);
+  if (!maybe_value) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property is invalid: value is " + value_str);

Review comment:
       Why are exceptions from these generic utilities categorized as `PROCESS_SCHEDULE_EXCEPTION`? I think `GENERAL_EXCEPTION` is the closest.

##########
File path: libminifi/include/utils/ProcessorConfigUtils.h
##########
@@ -0,0 +1,80 @@
+/**
+ * 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) {
+  std::string value;
+  if (!context->getProperty(property_name, value)) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property missing or invalid");
+  }
+  return value;
+}
+
+std::vector<std::string> listFromCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  std::string property_string;
+  context->getProperty(property_name, property_string);
+  return utils::StringUtils::splitAndTrim(property_string, ",");
+}
+
+std::vector<std::string> listFromRequiredCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  return utils::StringUtils::splitAndTrim(getRequiredPropertyOrThrow(context, property_name), ",");
+}
+
+bool parseBooleanPropertyOrThrow(core::ProcessContext* context, const std::string& property_name) {
+  bool value;
+  std::string value_str = getRequiredPropertyOrThrow(context, property_name);
+  utils::optional<bool> maybe_value = utils::StringUtils::toBool(value_str);
+  if (!maybe_value) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property is invalid: value is " + value_str);
+  }
+  return maybe_value.value();
+}
+
+std::chrono::milliseconds parseTimePropertyMSOrThrow(core::ProcessContext* context, const std::string& property_name) {
+  core::TimeUnit unit;
+  uint64_t time_value_ms;
+  if (!core::Property::StringToTime(getRequiredPropertyOrThrow(context, property_name), time_value_ms, unit) || !core::Property::ConvertTimeUnitToMS(time_value_ms, unit, time_value_ms)) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property missing or invalid");

Review comment:
       If this `throw` statement is hit, then the property must be present, because if it wasn't, then `getRequiredPropertyOrThrow` would throw instead. I'd consider updating the message to reflect that.

##########
File path: libminifi/include/utils/ProcessorConfigUtils.h
##########
@@ -0,0 +1,80 @@
+/**
+ * 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) {
+  std::string value;
+  if (!context->getProperty(property_name, value)) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property missing or invalid");
+  }
+  return value;
+}
+
+std::vector<std::string> listFromCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  std::string property_string;
+  context->getProperty(property_name, property_string);
+  return utils::StringUtils::splitAndTrim(property_string, ",");
+}
+
+std::vector<std::string> listFromRequiredCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  return utils::StringUtils::splitAndTrim(getRequiredPropertyOrThrow(context, property_name), ",");
+}
+
+bool parseBooleanPropertyOrThrow(core::ProcessContext* context, const std::string& property_name) {
+  bool value;
+  std::string value_str = getRequiredPropertyOrThrow(context, property_name);
+  utils::optional<bool> maybe_value = utils::StringUtils::toBool(value_str);
+  if (!maybe_value) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property is invalid: value is " + value_str);
+  }
+  return maybe_value.value();
+}
+
+std::chrono::milliseconds parseTimePropertyMSOrThrow(core::ProcessContext* context, const std::string& property_name) {
+  core::TimeUnit unit;
+  uint64_t time_value_ms;
+  if (!core::Property::StringToTime(getRequiredPropertyOrThrow(context, property_name), time_value_ms, unit) || !core::Property::ConvertTimeUnitToMS(time_value_ms, unit, time_value_ms)) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property missing or invalid");
+  }
+  return std::chrono::milliseconds(time_value_ms);
+}
+
+utils::optional<uint64_t> getOptionalUintProperty(core::ProcessContext* context, const std::string& property_name) {

Review comment:
       I suggest converting these functions to take `const core::ProcessContext&` so that they can not be misused with null pointers and they are const correct.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r528651838



##########
File path: libminifi/test/TestBase.h
##########
@@ -200,7 +199,7 @@ class LogTestController {
     }
     my_properties_->set("logger.root", "ERROR,ostream");
     my_properties_->set("logger." + core::getClassName<LogTestController>(), "INFO");
-    my_properties_->set("logger." + core::getClassName<logging::LoggerConfiguration>(), "DEBUG");
+    my_properties_->set("logger." + core::getClassName<logging::LoggerConfiguration>(), "INFO");

Review comment:
       This was added for hiding the `[Component] got sinks from ...`, flooding the test logs. If someone finds those log lines helpful, I am happy to restore this.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553806575



##########
File path: libminifi/include/utils/ProcessorConfigUtils.h
##########
@@ -0,0 +1,80 @@
+/**
+ * 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) {
+  std::string value;
+  if (!context->getProperty(property_name, value)) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property missing or invalid");
+  }
+  return value;
+}
+
+std::vector<std::string> listFromCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  std::string property_string;
+  context->getProperty(property_name, property_string);
+  return utils::StringUtils::splitAndTrim(property_string, ",");
+}
+
+std::vector<std::string> listFromRequiredCommaSeparatedProperty(const core::ProcessContext* context, const std::string& property_name) {
+  return utils::StringUtils::splitAndTrim(getRequiredPropertyOrThrow(context, property_name), ",");
+}
+
+bool parseBooleanPropertyOrThrow(core::ProcessContext* context, const std::string& property_name) {
+  bool value;
+  std::string value_str = getRequiredPropertyOrThrow(context, property_name);
+  utils::optional<bool> maybe_value = utils::StringUtils::toBool(value_str);
+  if (!maybe_value) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property is invalid: value is " + value_str);
+  }
+  return maybe_value.value();
+}
+
+std::chrono::milliseconds parseTimePropertyMSOrThrow(core::ProcessContext* context, const std::string& property_name) {
+  core::TimeUnit unit;
+  uint64_t time_value_ms;
+  if (!core::Property::StringToTime(getRequiredPropertyOrThrow(context, property_name), time_value_ms, unit) || !core::Property::ConvertTimeUnitToMS(time_value_ms, unit, time_value_ms)) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " property missing or invalid");
+  }
+  return std::chrono::milliseconds(time_value_ms);
+}
+
+utils::optional<uint64_t> getOptionalUintProperty(core::ProcessContext* context, const std::string& property_name) {

Review comment:
       Good comment! Updated.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553346870



##########
File path: extensions/librdkafka/tests/ConsumeKafkaTests.cpp
##########
@@ -0,0 +1,593 @@
+
+/**
+ * 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 "../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(), "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) {
+    reInitialize();
+
+    // Consumer chain
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+    std::shared_ptr<core::Processor> extract_text  = plan_->addProcessor("ExtractText", "extract_text", {success}, false);
+
+    // Set up connections
+    plan_->addConnection(consume_kafka, success, extract_text);
+    extract_text->setAutoTerminatedRelationships({success});
+
+    const auto bool_to_string = [] (const bool b) -> std::string { return b ? "true" : "false"; };
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::SecurityProtocol.getName(), security_protocol);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), topic_names);
+
+    optional_set_property(consume_kafka, ConsumeKafka::TopicNameFormat.getName(), topic_name_format);
+    optional_set_property(consume_kafka, ConsumeKafka::HonorTransactions.getName(), honor_transactions | utils::map(bool_to_string));
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+    optional_set_property(consume_kafka, ConsumeKafka::OffsetReset.getName(), offset_reset);
+    optional_set_property(consume_kafka, ConsumeKafka::KeyAttributeEncoding.getName(), key_attribute_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageDemarcator.getName(), message_demarcator);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageHeaderEncoding.getName(), message_header_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::HeadersToAddAsAttributes.getName(), headers_to_add_as_attributes);
+    optional_set_property(consume_kafka, ConsumeKafka::DuplicateHeaderHandling.getName(), duplicate_header_handling);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    plan_->setProperty(extract_text, ExtractText::Attribute.getName(), ATTRIBUTE_FOR_CAPTURING_CONTENT);
+
+    if (!expect_config_valid) {
+      const auto& message = messages_on_topic.front();
+      REQUIRE_THROWS(plan_->scheduleProcessor(consume_kafka));
+      return;
+    } else {
+      plan_->scheduleProcessors();
+    }
+
+    std::unique_ptr<rd_kafka_conf_t, utils::rd_kafka_conf_deleter> conf_;
+    std::unique_ptr<rd_kafka_t, utils::rd_kafka_consumer_deleter> consumer_;
+
+    const bool is_transactional = std::count(transaction_events.cbegin(), transaction_events.cend(), KafkaTestProducer::PublishEvent::TRANSACTION_START);
+    const bool transactions_committed = transaction_events.back() == KafkaTestProducer::PublishEvent::TRANSACTION_COMMIT;
+
+    KafkaTestProducer producer(kafka_brokers, PRODUCER_TOPIC, is_transactional);
+    producer.publish_messages_to_topic(messages_on_topic, TEST_MESSAGE_KEY, transaction_events, message_headers, message_header_encoding);
+
+
+    std::vector<std::shared_ptr<core::FlowFile>> flow_files_produced;
+    for (std::size_t num_expected_messages_processed = 0; num_expected_messages_processed < messages_on_topic.size(); num_expected_messages_processed += std::stoi(max_poll_records.value_or("1"))) {
+      plan_->increment_location();
+      if ((honor_transactions && false == honor_transactions.value()) || (is_transactional && !transactions_committed)) {
+        INFO("Non-committed messages received.");
+        REQUIRE(false == plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+        return;
+      }
+      {
+        SCOPED_INFO("ConsumeKafka timed out when waiting to receive the message published to the kafka broker.");
+        REQUIRE(plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+      }
+      std::size_t num_flow_files_produced = plan_->getNumFlowFileProducedByCurrentProcessor();
+      plan_->increment_location();
+      for (std::size_t times_extract_text_run = 0; times_extract_text_run < num_flow_files_produced; ++times_extract_text_run) {
+        plan_->runCurrentProcessor();  // ExtractText
+        std::shared_ptr<core::FlowFile> flow_file = plan_->getFlowFileProducedByCurrentProcessor();
+        for (const auto& exp_header : expect_header_attributes) {
+          SCOPED_INFO("ConsumeKafka did not produce the expected flowfile attribute from message header: " << exp_header.first << ".");
+          const auto header_attr_opt = flow_file->getAttribute(exp_header.first);
+          REQUIRE(header_attr_opt);
+          REQUIRE(exp_header.second == header_attr_opt.value().get());
+        }
+        {
+          SCOPED_INFO("Message key is missing or incorrect (potential encoding mismatch).");
+          REQUIRE(TEST_MESSAGE_KEY == decode_key(flow_file->getAttribute(ConsumeKafka::KAFKA_MESSAGE_KEY_ATTR).value().get(), key_attribute_encoding));
+          REQUIRE("1" == flow_file->getAttribute(ConsumeKafka::KAFKA_COUNT_ATTR).value().get());
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_OFFSET_ATTR));
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_PARTITION_ATTR));
+          REQUIRE(PRODUCER_TOPIC == flow_file->getAttribute(ConsumeKafka::KAFKA_TOPIC_ATTR).value().get());
+        }
+        flow_files_produced.emplace_back(std::move(flow_file));
+      }
+      plan_->reset_location();
+    }
+
+    const auto contentOrderOfFlowFile = [&] (const std::shared_ptr<core::FlowFile>& lhs, const std::shared_ptr<core::FlowFile>& rhs) {
+      return lhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() < rhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get();
+    };
+    {
+      SCOPED_INFO("The flowfiles generated by ConsumeKafka are invalid (probably nullptr).");
+      CHECK_NOTHROW(std::sort(flow_files_produced.begin(), flow_files_produced.end(), contentOrderOfFlowFile));
+    }
+    std::vector<std::string> sorted_split_messages = sort_and_split_messages(messages_on_topic, message_demarcator);
+    const auto flow_file_content_matches_message = [&] (const std::shared_ptr<core::FlowFile>& flowfile, const std::string message) {
+      return flowfile->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() == message;
+    };
+
+    logger_->log_debug("************");
+    std::string expected = "Expected: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      expected += sorted_split_messages[i] + ", ";
+    }
+    std::string   actual = "  Actual: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      actual += flow_files_produced[i]->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() + ", ";
+    }
+    logger_->log_debug("%s", expected.c_str());
+    logger_->log_debug("%s", actual.c_str());
+    logger_->log_debug("************");
+
+    INFO("The messages received by ConsumeKafka do not match those published");
+    REQUIRE(std::equal(flow_files_produced.begin(), flow_files_produced.end(), sorted_split_messages.begin(), flow_file_content_matches_message));
+  }
+};
+
+class ConsumeKafkaContinuousPublishingTest : public ConsumeKafkaTest {
+ public:
+  ConsumeKafkaContinuousPublishingTest() : ConsumeKafkaTest() {}
+  virtual ~ConsumeKafkaContinuousPublishingTest() {
+    logTestController_.reset();
+  }
+
+  void single_consumer_with_continuous_message_producing(
+      const uint64_t msg_periodicity_ms,
+      const std::string& kafka_brokers,
+      const optional<std::string>& group_id,
+      const optional<std::string>& max_poll_records,
+      const optional<std::string>& max_poll_time,
+      const optional<std::string>& session_timeout) {
+    reInitialize();
+
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+
+    plan_->setProperty(consume_kafka, "allow.auto.create.topics", "true", true);  // Seems like the topic tests work without this
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), PRODUCER_TOPIC);
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    consume_kafka->setAutoTerminatedRelationships({success});
+
+    KafkaTestProducer producer("localhost:9092", PRODUCER_TOPIC, /* transactional = */ false);
+
+    std::atomic_bool producer_loop_stop{ false };
+    std::size_t num_messages_sent = 0;  // When on C++14 make this an initialized lambda capture
+    const auto producer_loop = [&] {
+      std::this_thread::sleep_for(std::chrono::milliseconds(100));
+      while (!producer_loop_stop) {
+        producer.publish_messages_to_topic({ "Message after " + std::to_string(msg_periodicity_ms * num_messages_sent) + " ms"}, TEST_MESSAGE_KEY, { PUBLISH }, {}, {});
+        ++num_messages_sent;
+        std::this_thread::sleep_for(std::chrono::milliseconds(msg_periodicity_ms));
+      }
+      return num_messages_sent;
+    };
+
+    plan_->scheduleProcessors();
+
+    const auto get_time_property_ms = [] (const std::string& property_string) {
+      int64_t value;
+      org::apache::nifi::minifi::core::TimeUnit unit;
+      REQUIRE(org::apache::nifi::minifi::core::Property::StringToTime(property_string, value, unit));
+      int64_t value_as_ms;
+      REQUIRE(org::apache::nifi::minifi::core::Property::ConvertTimeUnitToMS(value, unit, value_as_ms));
+      return value_as_ms;
+    };
+
+    std::thread producer_thread(producer_loop);
+    CHECK_NOTHROW(plan_->runNextProcessor());
+    producer_loop_stop = true;
+    producer_thread.join();
+
+    std::size_t num_flow_files_produced = plan_->getNumFlowFileProducedByCurrentProcessor();
+
+    const uint64_t max_poll_time_ms = get_time_property_ms(max_poll_time.value_or(ConsumeKafka::DEFAULT_MAX_POLL_TIME));
+    const uint64_t max_poll_records_value = max_poll_records ? std::stoi(max_poll_records.value()) : ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+    const uint64_t exp_lower_bound = std::min(max_poll_time_ms / msg_periodicity_ms - 2, max_poll_records_value);
+    const uint64_t exp_upper_bound = std::min(max_poll_time_ms / msg_periodicity_ms + 2, max_poll_records_value);
+    logger_->log_debug("Max poll time: %d, Max poll records: %d, Exp. flowfiles produced: (min: %d, max: %d), actual: %d",
+        max_poll_time_ms, max_poll_records_value, exp_lower_bound, exp_upper_bound, num_flow_files_produced);
+
+    REQUIRE(exp_lower_bound <= num_flow_files_produced);
+    REQUIRE(num_flow_files_produced <= exp_upper_bound);
+  }
+};
+
+const std::string ConsumeKafkaTest::TEST_FILE_NAME_POSTFIX{ "target_kafka_message.txt" };
+const std::string ConsumeKafkaTest::TEST_MESSAGE_KEY{ "consume_kafka_test_key" };
+const std::string ConsumeKafkaTest::PRODUCER_TOPIC{ "ConsumeKafkaTest" };
+const std::string ConsumeKafkaTest::ATTRIBUTE_FOR_CAPTURING_CONTENT{ "flowfile_content" };
+const std::chrono::seconds ConsumeKafkaTest::MAX_CONSUMEKAFKA_POLL_TIME_SECONDS{ 5 };
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "ConsumeKafka parses and uses kafka topics.", "[ConsumeKafka][Kafka][Topic]") {
+  auto run_tests = [&] (const std::vector<std::string>& messages_on_topic, const std::string& topic_names, const optional<std::string>& topic_name_format) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", topic_names, topic_name_format, {}, "test_group_id", {}, {}, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({ "Ulysses",              "James Joyce"         }, "ConsumeKafkaTest",         {});
+  run_tests({ "The Great Gatsby",     "F. Scott Fitzgerald" }, "ConsumeKafkaTest",         ConsumeKafka::TOPIC_FORMAT_NAMES);
+  run_tests({ "War and Peace",        "Lev Tolstoy"         }, "a,b,c,ConsumeKafkaTest,d", ConsumeKafka::TOPIC_FORMAT_NAMES);
+  run_tests({ "Nineteen Eighty Four", "George Orwell"       }, "ConsumeKafkaTest",         ConsumeKafka::TOPIC_FORMAT_PATTERNS);
+  run_tests({ "Hamlet",               "William Shakespeare" }, "Cons[emu]*KafkaTest",      ConsumeKafka::TOPIC_FORMAT_PATTERNS);
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Offsets are reset to the latest when a consumer starts with non-processed messages.", "[ConsumeKafka][Kafka][OffsetReset]") {
+  auto run_tests = [&] (
+      const std::vector<std::string>& messages_on_topic,
+      const std::vector<KafkaTestProducer::PublishEvent>& transaction_events) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, transaction_events, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  KafkaTestProducer producer("localhost:9092", PRODUCER_TOPIC, false);
+  producer.publish_messages_to_topic({"Dummy messages", "that should be ignored", "due to offset reset on ConsumeKafka startup"}, TEST_MESSAGE_KEY, {PUBLISH, PUBLISH, PUBLISH}, {}, {});
+  run_tests({"Brave New World",  "Aldous Huxley"}, NON_TRANSACTIONAL_MESSAGES);
+  producer.publish_messages_to_topic({"Dummy messages", "that should be ignored", "due to offset reset on ConsumeKafka startup"}, TEST_MESSAGE_KEY, {PUBLISH, PUBLISH, PUBLISH}, {}, {});
+  run_tests({"Call of the Wild", "Jack London"}, NON_TRANSACTIONAL_MESSAGES);
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Key attribute is encoded according to the \"Key Attribute Encoding\" property.", "[ConsumeKafka][Kafka][KeyAttributeEncoding]") {
+  auto run_tests = [&] (const std::vector<std::string>& messages_on_topic, const optional<std::string>& key_attribute_encoding) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, key_attribute_encoding, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+
+  run_tests({ "The Odyssey",          "Ὅμηρος"                        }, {});
+  run_tests({ "Lolita",               "Владимир Владимирович Набоков" }, "utf-8");
+  run_tests({ "Crime and Punishment", "Фёдор Михайлович Достоевский"  }, "hex");
+  run_tests({ "Paradise Lost",        "John Milton"                   }, "hEX");
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Transactional behaviour is supported.", "[ConsumeKafka][Kafka][Transaction]") {
+  auto run_tests = [&] (const std::vector<std::string>& messages_on_topic, const std::vector<KafkaTestProducer::PublishEvent>& transaction_events, const optional<bool>& honor_transactions) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, transaction_events, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({  "Pride and Prejudice", "Jane Austen"      }, SINGLE_COMMITTED_TRANSACTION, {});
+  run_tests({                 "Dune", "Frank Herbert"    },    TWO_SEPARATE_TRANSACTIONS, {});
+  run_tests({      "The Black Sheep", "Honore De Balzac" },    NON_COMMITTED_TRANSACTION, {});
+  run_tests({     "Gospel of Thomas"                     },            COMMIT_AND_CANCEL, {});
+  run_tests({ "Operation Dark Heart"                     },            COMMIT_AND_CANCEL, true);
+  run_tests({               "Brexit"                     },            COMMIT_AND_CANCEL, false);
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Headers on consumed Kafka messages are extracted into attributes if requested on ConsumeKafka.", "[ConsumeKafka][Kafka][Batching][Headers]") {
+  auto run_tests = [&] (
+      const std::vector<std::string>& messages_on_topic,
+      const std::vector<std::pair<std::string, std::string>>& expect_header_attributes,
+      const std::vector<std::pair<std::string, std::string>>& message_headers,
+      const optional<std::string>& headers_to_add_as_attributes,
+      const optional<std::string>& duplicate_header_handling) {
+    single_consumer_with_plain_text_test(true, false, expect_header_attributes, messages_on_topic, NON_TRANSACTIONAL_MESSAGES, message_headers, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, headers_to_add_as_attributes, duplicate_header_handling, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({             "Homeland",   "R. A. Salvatore"},                                      {},             {{{"Contains dark elves"}, {"Yes"}}},         {},                    {});
+  run_tests({             "Magician",  "Raymond E. Feist"},               {{{"Rating"}, {"10/10"}}},                        {{{"Rating"}, {"10/10"}}}, {"Rating"},                    {});
+  run_tests({             "Mistborn", "Brandon Sanderson"},               {{{"Metal"}, {"Copper"}}}, {{{"Metal"}, {"Copper"}}, {{"Metal"}, {"Iron"}}},  {"Metal"},            KEEP_FIRST);
+  run_tests({             "Mistborn", "Brandon Sanderson"},                 {{{"Metal"}, {"Iron"}}}, {{{"Metal"}, {"Copper"}}, {{"Metal"}, {"Iron"}}},  {"Metal"},           KEEP_LATEST);
+  run_tests({             "Mistborn", "Brandon Sanderson"},         {{{"Metal"}, {"Copper, Iron"}}}, {{{"Metal"}, {"Copper"}}, {{"Metal"}, {"Iron"}}},  {"Metal"}, COMMA_SEPARATED_MERGE);
+  run_tests({"The Lord of the Rings",  "J. R. R. Tolkien"}, {{{"Parts"}, {"First, second, third"}}},          {{{"Parts"}, {"First, second, third"}}},  {"Parts"},                    {});
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Messages are separated into multiple flowfiles if the message demarcator is present in the message.", "[ConsumeKafka][Kafka][MessageDemarcator]") {
+  auto run_tests = [&] (
+      const std::vector<std::string>& messages_on_topic,
+      const optional<std::string>& message_demarcator) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, message_demarcator, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({"Barbapapa", "Anette Tison and Talus Taylor"}, "a");
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "The maximum poll records allows ConsumeKafka to combine multiple messages into a single flowfile.", "[ConsumeKafka][Kafka][Batching][MaxPollRecords]") {
+  auto run_tests = [&] (
+      const std::vector<std::string>& messages_on_topic,
+      const std::vector<KafkaTestProducer::PublishEvent>& transaction_events,
+      const optional<std::string>& max_poll_records) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, transaction_events, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, {}, {}, max_poll_records, "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({"The Count of Monte Cristo", "Alexandre Dumas"}, NON_TRANSACTIONAL_MESSAGES, "2");
+
+  const std::vector<std::string> content {
+      "Make const member functions thread safe",
+      "Understand special member function generation",
+      "Use std::unique_ptr for exclusive-ownership resource management",
+      "Use std::shared_ptr for shared-ownership resource management",
+      "Use std::weak_ptr for std::shared_ptr-like pointers that can dangle",
+      "Prefer std::make_unique and std::make_shared to direct use of new",
+      "When using the Pimpl Idiom, define special member functions inthe implementation file",
+      "Understand std::move and std::forward",
+      "Distinguish universal references from rvalue references",
+      "Use std::move on rvalue references, std::forward on universal references",
+      "Avoid overloading on universal references",
+      "Familiarize yourself with alternatives to overloading on universal references",
+      "Understand reference collapsing",
+      "Assume that move operations are not present, not cheap, and not used",
+      "Familiarize yourself with perfect forwarding failure cases",
+      "Avoid default capture modes",
+      "Use init capture to move objects into closures",
+      "Use decltype on auto&& parameters to std::forward them",
+      "Prefer lambdas to std::bind",
+      "Prefer task-based programming to thread-based" };
+  const std::vector<KafkaTestProducer::PublishEvent> transaction_events(content.size(), PUBLISH);
+  run_tests(content, transaction_events, "5");
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Non-plain text security context throws scheduling exceptions.", "[ConsumeKafka][Kafka][SecurityProtocol]") {
+  single_consumer_with_plain_text_test(false, false, {}, { "Miyamoto Musashi", "Eiji Yoshikawa" }, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", ConsumeKafka::SECURITY_PROTOCOL_SSL, "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Acceptable values for message header and key attribute encoding are \"UTF-8\" and \"hex\".", "[ConsumeKafka][Kafka][InvalidEncoding]") {
+  single_consumer_with_plain_text_test(false, false, {}, {                           "Shogun", "James Clavell" }, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, "UTF-32", {},       {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  single_consumer_with_plain_text_test(false, false, {}, { "Alice's Adventures in Wonderland", "Lewis Carroll" }, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {},       {}, {}, "UTF-32", {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+}
+
+TEST_CASE_METHOD(ConsumeKafkaContinuousPublishingTest, "ConsumeKafka can spend no more time polling than allowed in the maximum poll time property.", "[ConsumeKafka][Kafka][Batching][MaxPollTime]") {
+  auto run_tests = [&] (
+      const uint64_t msg_periodicity_ms,
+      const optional<std::string>& max_poll_records,
+      const optional<std::string>& max_poll_time,
+      const optional<std::string>& session_timeout) {
+    single_consumer_with_continuous_message_producing(msg_periodicity_ms, "localhost:9092", "test_group_id", max_poll_records, max_poll_time, session_timeout);
+  };
+  // For some reason, a session time-out of a few seconds does not work at all, 10 seconds seems to be stable
+  run_tests(300, "20", "3 seconds", "10000 ms");

Review comment:
       This does not quite add up to 10 seconds:
   ```bash
   ./extensions/librdkafka/tests/ConsumeKafkaTests "[MaxPollTime]"  0.04s user 0.00s system 0% cpu 6.122 total
   ```
   However, it is a good observation, it takes quite a while to run all the tests even without setting up a broker:
   ```bash
   ./extensions/librdkafka/tests/ConsumeKafkaTests  0.45s user 0.12s system 0% cpu 1:22.58 total
   ```
   The plan with this is to convert it to a docker-based integration test where time should not be such a big concern.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r556352310



##########
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;
+    // }
+    rd_kafka_destroy(ptr);
+  }
+};
+
+struct rd_kafka_consumer_deleter {
+  void operator()(rd_kafka_t* ptr) const noexcept {
+    rd_kafka_consumer_close(ptr);
+    rd_kafka_destroy(ptr);
+  }
+};
+
+struct rd_kafka_topic_partition_list_deleter {
+  void operator()(rd_kafka_topic_partition_list_t* ptr) const noexcept { rd_kafka_topic_partition_list_destroy(ptr); }
+};
+
+struct rd_kafka_topic_conf_deleter {
+  void operator()(rd_kafka_topic_conf_t* ptr) const noexcept { rd_kafka_topic_conf_destroy(ptr); }
+};
+struct rd_kafka_topic_deleter {
+  void operator()(rd_kafka_topic_t* ptr) const noexcept { rd_kafka_topic_destroy(ptr); }
+};
+
+struct rd_kafka_message_deleter {
+  void operator()(rd_kafka_message_t* ptr) const noexcept { rd_kafka_message_destroy(ptr); }
+};
+
+struct rd_kafka_headers_deleter {
+  void operator()(rd_kafka_headers_t* ptr) const noexcept { rd_kafka_headers_destroy(ptr); }
+};
+
+template <typename T>
+void kafka_headers_for_each(const rd_kafka_headers_t* headers, T key_value_handle) {
+  const char *key;  // Null terminated, not to be freed
+  const void *value;
+  std::size_t size;
+  for (std::size_t i = 0; RD_KAFKA_RESP_ERR_NO_ERROR == rd_kafka_header_get_all(headers, i, &key, &value, &size); ++i) {
+    key_value_handle(std::string(key), std::string(static_cast<const char*>(value), size));

Review comment:
       We can avoid allocation in every cycle by passing the raw `const char*` down to the consumer.

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,117 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error: " + error_msg);
+  }
+}
+
+void print_topics_list(logging::Logger& logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger.log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, logging::Logger& logger) {

Review comment:
       Did you consider extracting the message formatting part? It may be useful outside of logging.

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,117 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error: " + error_msg);
+  }
+}
+
+void print_topics_list(logging::Logger& logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger.log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, logging::Logger& logger) {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    const std::string error_msg = "ConsumeKafka: received error message from broker. Librdkafka error msg: " + std::string(rd_kafka_err2str(rkmessage->err));
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, error_msg);
+  }
+  std::string topicName = rd_kafka_topic_name(rkmessage->rkt);
+  std::string message(reinterpret_cast<char*>(rkmessage->payload), rkmessage->len);
+  const char* key = reinterpret_cast<const char*>(rkmessage->key);
+  const std::size_t key_len = rkmessage->key_len;
+  rd_kafka_timestamp_type_t tstype;
+  int64_t timestamp;
+  timestamp = rd_kafka_message_timestamp(rkmessage, &tstype);
+  const char *tsname = "?";
+  if (tstype != RD_KAFKA_TIMESTAMP_NOT_AVAILABLE) {
+    if (tstype == RD_KAFKA_TIMESTAMP_CREATE_TIME) {
+      tsname = "create time";
+    } else if (tstype == RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME) {
+      tsname = "log append time";
+    }
+  }

Review comment:
       The outer `if` is unnecessary.
   ```suggestion
     if (tstype == RD_KAFKA_TIMESTAMP_CREATE_TIME) {
       tsname = "create time";
     } else if (tstype == RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME) {
       tsname = "log append time";
     }
   ```

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,569 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));

Review comment:
       I wouldn't call `fmt` a framework and while `join_pack`'s implementation is complex, its interface couldn't be simpler. I won't reject the PR if you leave it like this.

##########
File path: extensions/librdkafka/tests/ConsumeKafkaTests.cpp
##########
@@ -0,0 +1,590 @@
+
+/**
+ * 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 "../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(), "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 = "Failed to create Kafka producer" + std::string{ errstr.data() };
+      throw std::runtime_error(error_msg);
+    }
+
+    // The last argument is a config here, but it is already owned by the producer. 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> CANCELLED_TRANSACTION        { 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) {
+    reInitialize();
+
+    // Consumer chain
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+    std::shared_ptr<core::Processor> extract_text  = plan_->addProcessor("ExtractText", "extract_text", {success}, false);
+
+    // Set up connections
+    plan_->addConnection(consume_kafka, success, extract_text);
+    extract_text->setAutoTerminatedRelationships({success});
+
+    const auto bool_to_string = [] (const bool b) -> std::string { return b ? "true" : "false"; };
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::SecurityProtocol.getName(), security_protocol);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), topic_names);
+
+    optional_set_property(consume_kafka, ConsumeKafka::TopicNameFormat.getName(), topic_name_format);
+    optional_set_property(consume_kafka, ConsumeKafka::HonorTransactions.getName(), honor_transactions | utils::map(bool_to_string));
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+    optional_set_property(consume_kafka, ConsumeKafka::OffsetReset.getName(), offset_reset);
+    optional_set_property(consume_kafka, ConsumeKafka::KeyAttributeEncoding.getName(), key_attribute_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageDemarcator.getName(), message_demarcator);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageHeaderEncoding.getName(), message_header_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::HeadersToAddAsAttributes.getName(), headers_to_add_as_attributes);
+    optional_set_property(consume_kafka, ConsumeKafka::DuplicateHeaderHandling.getName(), duplicate_header_handling);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    plan_->setProperty(extract_text, ExtractText::Attribute.getName(), ATTRIBUTE_FOR_CAPTURING_CONTENT);
+
+    if (!expect_config_valid) {
+      REQUIRE_THROWS(plan_->scheduleProcessor(consume_kafka));
+      return;
+    } else {
+      plan_->scheduleProcessors();
+    }
+
+    std::unique_ptr<rd_kafka_conf_t, utils::rd_kafka_conf_deleter> conf_;
+    std::unique_ptr<rd_kafka_t, utils::rd_kafka_consumer_deleter> consumer_;
+
+    const bool is_transactional = std::count(transaction_events.cbegin(), transaction_events.cend(), KafkaTestProducer::PublishEvent::TRANSACTION_START);
+    const bool transactions_committed = transaction_events.back() == KafkaTestProducer::PublishEvent::TRANSACTION_COMMIT;
+
+    KafkaTestProducer producer(kafka_brokers, PRODUCER_TOPIC, is_transactional);
+    producer.publish_messages_to_topic(messages_on_topic, TEST_MESSAGE_KEY, transaction_events, message_headers, message_header_encoding);
+
+
+    std::vector<std::shared_ptr<core::FlowFile>> flow_files_produced;
+    for (std::size_t num_expected_messages_processed = 0; num_expected_messages_processed < messages_on_topic.size(); num_expected_messages_processed += std::stoi(max_poll_records.value_or("1"))) {
+      plan_->increment_location();
+      if ((honor_transactions && false == honor_transactions.value()) || (is_transactional && !transactions_committed)) {
+        INFO("Non-committed messages received.");
+        REQUIRE(false == plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+        return;
+      }
+      {
+        SCOPED_INFO("ConsumeKafka timed out when waiting to receive the message published to the kafka broker.");
+        REQUIRE(plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+      }
+      std::size_t num_flow_files_produced = plan_->getNumFlowFileProducedByCurrentProcessor();
+      plan_->increment_location();
+      for (std::size_t times_extract_text_run = 0; times_extract_text_run < num_flow_files_produced; ++times_extract_text_run) {
+        plan_->runCurrentProcessor();  // ExtractText
+        std::shared_ptr<core::FlowFile> flow_file = plan_->getFlowFileProducedByCurrentProcessor();
+        for (const auto& exp_header : expect_header_attributes) {
+          SCOPED_INFO("ConsumeKafka did not produce the expected flowfile attribute from message header: " << exp_header.first << ".");
+          const auto header_attr_opt = flow_file->getAttribute(exp_header.first);
+          REQUIRE(header_attr_opt);
+          REQUIRE(exp_header.second == header_attr_opt.value());
+        }
+        {
+          SCOPED_INFO("Message key is missing or incorrect (potential encoding mismatch).");
+          REQUIRE(TEST_MESSAGE_KEY == decode_key(flow_file->getAttribute(ConsumeKafka::KAFKA_MESSAGE_KEY_ATTR).value(), key_attribute_encoding));
+          REQUIRE("1" == flow_file->getAttribute(ConsumeKafka::KAFKA_COUNT_ATTR).value());
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_OFFSET_ATTR));
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_PARTITION_ATTR));
+          REQUIRE(PRODUCER_TOPIC == flow_file->getAttribute(ConsumeKafka::KAFKA_TOPIC_ATTR).value());
+        }
+        flow_files_produced.emplace_back(std::move(flow_file));
+      }
+      plan_->reset_location();
+    }
+
+    const auto contentOrderOfFlowFile = [&] (const std::shared_ptr<core::FlowFile>& lhs, const std::shared_ptr<core::FlowFile>& rhs) {
+      return lhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value() < rhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value();
+    };
+    {
+      SCOPED_INFO("The flowfiles generated by ConsumeKafka are invalid (probably nullptr).");
+      REQUIRE_NOTHROW(std::sort(flow_files_produced.begin(), flow_files_produced.end(), contentOrderOfFlowFile));
+    }
+    std::vector<std::string> sorted_split_messages = sort_and_split_messages(messages_on_topic, message_demarcator);
+    const auto flow_file_content_matches_message = [&] (const std::shared_ptr<core::FlowFile>& flowfile, const std::string message) {
+      return flowfile->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value() == message;
+    };
+
+    logger_->log_debug("************");
+    std::string expected = "Expected: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      expected += sorted_split_messages[i] + ", ";
+    }
+    std::string   actual = "  Actual: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      actual += flow_files_produced[i]->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value() + ", ";
+    }
+    logger_->log_debug("%s", expected.c_str());
+    logger_->log_debug("%s", actual.c_str());
+    logger_->log_debug("************");
+
+    INFO("The messages received by ConsumeKafka do not match those published");
+    REQUIRE(std::equal(flow_files_produced.begin(), flow_files_produced.end(), sorted_split_messages.begin(), flow_file_content_matches_message));
+  }
+};
+
+class ConsumeKafkaContinuousPublishingTest : public ConsumeKafkaTest {
+ public:
+  ConsumeKafkaContinuousPublishingTest() : ConsumeKafkaTest() {}
+  virtual ~ConsumeKafkaContinuousPublishingTest() {
+    logTestController_.reset();
+  }
+
+  void single_consumer_with_continuous_message_producing(
+      const uint64_t msg_periodicity_ms,
+      const std::string& kafka_brokers,
+      const optional<std::string>& group_id,
+      const optional<std::string>& max_poll_records,
+      const optional<std::string>& max_poll_time,
+      const optional<std::string>& session_timeout) {
+    reInitialize();
+
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+
+    plan_->setProperty(consume_kafka, "allow.auto.create.topics", "true", true);  // Seems like the topic tests work without this
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), PRODUCER_TOPIC);
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    consume_kafka->setAutoTerminatedRelationships({success});
+
+    KafkaTestProducer producer("localhost:9092", PRODUCER_TOPIC, /* transactional = */ false);
+
+    std::atomic_bool producer_loop_stop{ false };
+    const auto producer_loop = [&] {
+      std::size_t num_messages_sent = 0;
+      std::this_thread::sleep_for(std::chrono::milliseconds(100));
+      while (!producer_loop_stop) {
+        producer.publish_messages_to_topic({ "Message after " + std::to_string(msg_periodicity_ms * ++num_messages_sent) + " ms"}, TEST_MESSAGE_KEY, { PUBLISH }, {}, {});
+        std::this_thread::sleep_for(std::chrono::milliseconds(msg_periodicity_ms));
+      }
+    };
+
+    plan_->scheduleProcessors();
+
+    const auto get_time_property_ms = [] (const std::string& property_string) {
+      int64_t value;
+      org::apache::nifi::minifi::core::TimeUnit unit;
+      REQUIRE(org::apache::nifi::minifi::core::Property::StringToTime(property_string, value, unit));
+      int64_t value_as_ms;
+      REQUIRE(org::apache::nifi::minifi::core::Property::ConvertTimeUnitToMS(value, unit, value_as_ms));
+      return value_as_ms;
+    };
+
+    std::thread producer_thread(producer_loop);
+    CHECK_NOTHROW(plan_->runNextProcessor());
+    producer_loop_stop = true;
+    producer_thread.join();
+
+    std::size_t num_flow_files_produced = plan_->getNumFlowFileProducedByCurrentProcessor();
+
+    const uint64_t max_poll_time_ms = get_time_property_ms(max_poll_time.value_or(ConsumeKafka::DEFAULT_MAX_POLL_TIME));
+    const uint64_t max_poll_records_value = max_poll_records ? std::stoi(max_poll_records.value()) : ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+    const uint64_t exp_lower_bound = std::min(max_poll_time_ms / msg_periodicity_ms - 2, max_poll_records_value);
+    const uint64_t exp_upper_bound = std::min(max_poll_time_ms / msg_periodicity_ms + 2, max_poll_records_value);
+    logger_->log_debug("Max poll time: %d, Max poll records: %d, Exp. flowfiles produced: (min: %d, max: %d), actual: %d",
+        max_poll_time_ms, max_poll_records_value, exp_lower_bound, exp_upper_bound, num_flow_files_produced);
+
+    REQUIRE(exp_lower_bound <= num_flow_files_produced);
+    REQUIRE(num_flow_files_produced <= exp_upper_bound);
+  }
+};
+
+const std::string ConsumeKafkaTest::TEST_FILE_NAME_POSTFIX{ "target_kafka_message.txt" };
+const std::string ConsumeKafkaTest::TEST_MESSAGE_KEY{ "consume_kafka_test_key" };
+const std::string ConsumeKafkaTest::PRODUCER_TOPIC{ "ConsumeKafkaTest" };
+const std::string ConsumeKafkaTest::ATTRIBUTE_FOR_CAPTURING_CONTENT{ "flowfile_content" };
+const std::chrono::seconds ConsumeKafkaTest::MAX_CONSUMEKAFKA_POLL_TIME_SECONDS{ 5 };
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "ConsumeKafka parses and uses kafka topics.", "[ConsumeKafka][Kafka][Topic]") {
+  auto run_tests = [&] (const std::vector<std::string>& messages_on_topic, const std::string& topic_names, const optional<std::string>& topic_name_format) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", topic_names, topic_name_format, {}, "test_group_id", {}, {}, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({ "Ulysses",              "James Joyce"         }, "ConsumeKafkaTest",         {});
+  run_tests({ "The Great Gatsby",     "F. Scott Fitzgerald" }, "ConsumeKafkaTest",         ConsumeKafka::TOPIC_FORMAT_NAMES);
+  run_tests({ "War and Peace",        "Lev Tolstoy"         }, "a,b,c,ConsumeKafkaTest,d", ConsumeKafka::TOPIC_FORMAT_NAMES);
+  run_tests({ "Nineteen Eighty Four", "George Orwell"       }, "ConsumeKafkaTest",         ConsumeKafka::TOPIC_FORMAT_PATTERNS);
+  run_tests({ "Hamlet",               "William Shakespeare" }, "Cons[emu]*KafkaTest",      ConsumeKafka::TOPIC_FORMAT_PATTERNS);
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Offsets are reset to the latest when a consumer starts with non-processed messages.", "[ConsumeKafka][Kafka][OffsetReset]") {
+  auto run_tests = [&] (
+      const std::vector<std::string>& messages_on_topic,
+      const std::vector<KafkaTestProducer::PublishEvent>& transaction_events) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, transaction_events, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  KafkaTestProducer producer("localhost:9092", PRODUCER_TOPIC, false);
+  producer.publish_messages_to_topic({"Dummy messages", "that should be ignored", "due to offset reset on ConsumeKafka startup"}, TEST_MESSAGE_KEY, {PUBLISH, PUBLISH, PUBLISH}, {}, {});
+  run_tests({"Brave New World",  "Aldous Huxley"}, NON_TRANSACTIONAL_MESSAGES);
+  producer.publish_messages_to_topic({"Dummy messages", "that should be ignored", "due to offset reset on ConsumeKafka startup"}, TEST_MESSAGE_KEY, {PUBLISH, PUBLISH, PUBLISH}, {}, {});
+  run_tests({"Call of the Wild", "Jack London"}, NON_TRANSACTIONAL_MESSAGES);
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Key attribute is encoded according to the \"Key Attribute Encoding\" property.", "[ConsumeKafka][Kafka][KeyAttributeEncoding]") {
+  auto run_tests = [&] (const std::vector<std::string>& messages_on_topic, const optional<std::string>& key_attribute_encoding) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, key_attribute_encoding, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+
+  run_tests({ "The Odyssey",          "Ὅμηρος"                        }, {});
+  run_tests({ "Lolita",               "Владимир Владимирович Набоков" }, "utf-8");
+  run_tests({ "Crime and Punishment", "Фёдор Михайлович Достоевский"  }, "hex");
+  run_tests({ "Paradise Lost",        "John Milton"                   }, "hEX");
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Transactional behaviour is supported.", "[ConsumeKafka][Kafka][Transaction]") {
+  auto run_tests = [&] (const std::vector<std::string>& messages_on_topic, const std::vector<KafkaTestProducer::PublishEvent>& transaction_events, const optional<bool>& honor_transactions) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, transaction_events, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({  "Pride and Prejudice", "Jane Austen"      }, SINGLE_COMMITTED_TRANSACTION, {});
+  run_tests({                 "Dune", "Frank Herbert"    },    TWO_SEPARATE_TRANSACTIONS, {});
+  run_tests({      "The Black Sheep", "Honore De Balzac" },    NON_COMMITTED_TRANSACTION, {});
+  run_tests({     "Gospel of Thomas"                     },        CANCELLED_TRANSACTION, {});
+  run_tests({ "Operation Dark Heart"                     },        CANCELLED_TRANSACTION, true);
+  run_tests({               "Brexit"                     },        CANCELLED_TRANSACTION, false);
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Headers on consumed Kafka messages are extracted into attributes if requested on ConsumeKafka.", "[ConsumeKafka][Kafka][Headers]") {
+  auto run_tests = [&] (
+      const std::vector<std::string>& messages_on_topic,
+      const std::vector<std::pair<std::string, std::string>>& expect_header_attributes,
+      const std::vector<std::pair<std::string, std::string>>& message_headers,
+      const optional<std::string>& headers_to_add_as_attributes,
+      const optional<std::string>& duplicate_header_handling) {
+    single_consumer_with_plain_text_test(true, false, expect_header_attributes, messages_on_topic, NON_TRANSACTIONAL_MESSAGES, message_headers, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, headers_to_add_as_attributes, duplicate_header_handling, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({             "Homeland",   "R. A. Salvatore"},                                      {},             {{{"Contains dark elves"}, {"Yes"}}},         {},                    {});
+  run_tests({             "Magician",  "Raymond E. Feist"},               {{{"Rating"}, {"10/10"}}},                        {{{"Rating"}, {"10/10"}}}, {"Rating"},                    {});
+  run_tests({             "Mistborn", "Brandon Sanderson"},               {{{"Metal"}, {"Copper"}}}, {{{"Metal"}, {"Copper"}}, {{"Metal"}, {"Iron"}}},  {"Metal"},            KEEP_FIRST);
+  run_tests({             "Mistborn", "Brandon Sanderson"},                 {{{"Metal"}, {"Iron"}}}, {{{"Metal"}, {"Copper"}}, {{"Metal"}, {"Iron"}}},  {"Metal"},           KEEP_LATEST);
+  run_tests({             "Mistborn", "Brandon Sanderson"},         {{{"Metal"}, {"Copper, Iron"}}}, {{{"Metal"}, {"Copper"}}, {{"Metal"}, {"Iron"}}},  {"Metal"}, COMMA_SEPARATED_MERGE);
+  run_tests({"The Lord of the Rings",  "J. R. R. Tolkien"}, {{{"Parts"}, {"First, second, third"}}},          {{{"Parts"}, {"First, second, third"}}},  {"Parts"},                    {});
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Messages are separated into multiple flowfiles if the message demarcator is present in the message.", "[ConsumeKafka][Kafka][MessageDemarcator]") {
+  auto run_tests = [&] (
+      const std::vector<std::string>& messages_on_topic,
+      const optional<std::string>& message_demarcator) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, message_demarcator, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({"Barbapapa", "Anette Tison and Talus Taylor"}, "a");
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "The maximum poll records allows ConsumeKafka to combine multiple messages into a single flowfile.", "[ConsumeKafka][Kafka][Batching][MaxPollRecords]") {
+  auto run_tests = [&] (
+      const std::vector<std::string>& messages_on_topic,
+      const std::vector<KafkaTestProducer::PublishEvent>& transaction_events,
+      const optional<std::string>& max_poll_records) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, transaction_events, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, {}, {}, max_poll_records, "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({"The Count of Monte Cristo", "Alexandre Dumas"}, NON_TRANSACTIONAL_MESSAGES, "2");
+
+  const std::vector<std::string> content {
+      "Make const member functions thread safe",
+      "Understand special member function generation",
+      "Use std::unique_ptr for exclusive-ownership resource management",
+      "Use std::shared_ptr for shared-ownership resource management",
+      "Use std::weak_ptr for std::shared_ptr-like pointers that can dangle",
+      "Prefer std::make_unique and std::make_shared to direct use of new",
+      "When using the Pimpl Idiom, define special member functions inthe implementation file",
+      "Understand std::move and std::forward",
+      "Distinguish universal references from rvalue references",
+      "Use std::move on rvalue references, std::forward on universal references",
+      "Avoid overloading on universal references",
+      "Familiarize yourself with alternatives to overloading on universal references",
+      "Understand reference collapsing",
+      "Assume that move operations are not present, not cheap, and not used",
+      "Familiarize yourself with perfect forwarding failure cases",
+      "Avoid default capture modes",
+      "Use init capture to move objects into closures",
+      "Use decltype on auto&& parameters to std::forward them",
+      "Prefer lambdas to std::bind",
+      "Prefer task-based programming to thread-based" };
+  const std::vector<KafkaTestProducer::PublishEvent> transaction_events(content.size(), PUBLISH);
+  run_tests(content, transaction_events, "5");
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Non-plain text security context throws scheduling exceptions.", "[ConsumeKafka][Kafka][SecurityProtocol]") {
+  single_consumer_with_plain_text_test(false, false, {}, { "Miyamoto Musashi", "Eiji Yoshikawa" }, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", ConsumeKafka::SECURITY_PROTOCOL_SSL, "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Acceptable values for message header and key attribute encoding are \"UTF-8\" and \"hex\".", "[ConsumeKafka][Kafka][InvalidEncoding]") {
+  single_consumer_with_plain_text_test(false, false, {}, {                           "Shogun", "James Clavell" }, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, "UTF-32", {},       {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  single_consumer_with_plain_text_test(false, false, {}, { "Alice's Adventures in Wonderland", "Lewis Carroll" }, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {},       {}, {}, "UTF-32", {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+}
+
+TEST_CASE_METHOD(ConsumeKafkaContinuousPublishingTest, "ConsumeKafka can spend no more time polling than allowed in the maximum poll time property.", "[ConsumeKafka][Kafka][Batching][MaxPollTime]") {
+  auto run_tests = [&] (
+      const uint64_t msg_periodicity_ms,
+      const optional<std::string>& max_poll_records,
+      const optional<std::string>& max_poll_time,
+      const optional<std::string>& session_timeout) {
+    single_consumer_with_continuous_message_producing(msg_periodicity_ms, "localhost:9092", "test_group_id", max_poll_records, max_poll_time, session_timeout);
+  };
+  // For some reason, a session time-out of a few seconds does not work at all, 10 seconds seems to be stable
+  run_tests(300, "20", "3 seconds", "10000 ms");
+  // Running multiple tests does not work properly here. For some reason, producing messages
+  // while a rebalance is triggered causes this error, and a blocked poll when new
+  // messages are produced:
+  //     Group "test_group_id" heartbeat error response in state up (join state wait-revoke-rebalance_cb, 1 partition(s) assigned): Broker: Group rebalance in progress
+  //
+  //  I tried adding a wait time for more than "session.timeout.ms" inbetween tests, but it was not sufficiend

Review comment:
       typo: `s/sufficiend/sufficient/`

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,569 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));
+  }
+  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::steady_clock::now();
+  auto elapsed = std::chrono::steady_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());

Review comment:
       not yet removed

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,117 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error: " + error_msg);
+  }
+}
+
+void print_topics_list(logging::Logger& logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger.log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld",

Review comment:
       `offset` is `int64_t`, not `long long int`. The format specifier should be `"%" PRId64`.

##########
File path: extensions/librdkafka/tests/ConsumeKafkaTests.cpp
##########
@@ -0,0 +1,590 @@
+
+/**
+ * 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 "../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(), "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 = "Failed to create Kafka producer" + std::string{ errstr.data() };
+      throw std::runtime_error(error_msg);
+    }
+
+    // The last argument is a config here, but it is already owned by the producer. 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> CANCELLED_TRANSACTION        { 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) {
+    reInitialize();
+
+    // Consumer chain
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+    std::shared_ptr<core::Processor> extract_text  = plan_->addProcessor("ExtractText", "extract_text", {success}, false);
+
+    // Set up connections
+    plan_->addConnection(consume_kafka, success, extract_text);
+    extract_text->setAutoTerminatedRelationships({success});
+
+    const auto bool_to_string = [] (const bool b) -> std::string { return b ? "true" : "false"; };
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::SecurityProtocol.getName(), security_protocol);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), topic_names);
+
+    optional_set_property(consume_kafka, ConsumeKafka::TopicNameFormat.getName(), topic_name_format);
+    optional_set_property(consume_kafka, ConsumeKafka::HonorTransactions.getName(), honor_transactions | utils::map(bool_to_string));
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+    optional_set_property(consume_kafka, ConsumeKafka::OffsetReset.getName(), offset_reset);
+    optional_set_property(consume_kafka, ConsumeKafka::KeyAttributeEncoding.getName(), key_attribute_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageDemarcator.getName(), message_demarcator);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageHeaderEncoding.getName(), message_header_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::HeadersToAddAsAttributes.getName(), headers_to_add_as_attributes);
+    optional_set_property(consume_kafka, ConsumeKafka::DuplicateHeaderHandling.getName(), duplicate_header_handling);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    plan_->setProperty(extract_text, ExtractText::Attribute.getName(), ATTRIBUTE_FOR_CAPTURING_CONTENT);
+
+    if (!expect_config_valid) {
+      REQUIRE_THROWS(plan_->scheduleProcessor(consume_kafka));
+      return;
+    } else {
+      plan_->scheduleProcessors();
+    }
+
+    std::unique_ptr<rd_kafka_conf_t, utils::rd_kafka_conf_deleter> conf_;
+    std::unique_ptr<rd_kafka_t, utils::rd_kafka_consumer_deleter> consumer_;
+
+    const bool is_transactional = std::count(transaction_events.cbegin(), transaction_events.cend(), KafkaTestProducer::PublishEvent::TRANSACTION_START);
+    const bool transactions_committed = transaction_events.back() == KafkaTestProducer::PublishEvent::TRANSACTION_COMMIT;
+
+    KafkaTestProducer producer(kafka_brokers, PRODUCER_TOPIC, is_transactional);
+    producer.publish_messages_to_topic(messages_on_topic, TEST_MESSAGE_KEY, transaction_events, message_headers, message_header_encoding);
+
+
+    std::vector<std::shared_ptr<core::FlowFile>> flow_files_produced;
+    for (std::size_t num_expected_messages_processed = 0; num_expected_messages_processed < messages_on_topic.size(); num_expected_messages_processed += std::stoi(max_poll_records.value_or("1"))) {
+      plan_->increment_location();
+      if ((honor_transactions && false == honor_transactions.value()) || (is_transactional && !transactions_committed)) {
+        INFO("Non-committed messages received.");
+        REQUIRE(false == plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+        return;
+      }
+      {
+        SCOPED_INFO("ConsumeKafka timed out when waiting to receive the message published to the kafka broker.");
+        REQUIRE(plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+      }
+      std::size_t num_flow_files_produced = plan_->getNumFlowFileProducedByCurrentProcessor();
+      plan_->increment_location();
+      for (std::size_t times_extract_text_run = 0; times_extract_text_run < num_flow_files_produced; ++times_extract_text_run) {
+        plan_->runCurrentProcessor();  // ExtractText
+        std::shared_ptr<core::FlowFile> flow_file = plan_->getFlowFileProducedByCurrentProcessor();
+        for (const auto& exp_header : expect_header_attributes) {
+          SCOPED_INFO("ConsumeKafka did not produce the expected flowfile attribute from message header: " << exp_header.first << ".");
+          const auto header_attr_opt = flow_file->getAttribute(exp_header.first);
+          REQUIRE(header_attr_opt);
+          REQUIRE(exp_header.second == header_attr_opt.value());
+        }
+        {
+          SCOPED_INFO("Message key is missing or incorrect (potential encoding mismatch).");
+          REQUIRE(TEST_MESSAGE_KEY == decode_key(flow_file->getAttribute(ConsumeKafka::KAFKA_MESSAGE_KEY_ATTR).value(), key_attribute_encoding));
+          REQUIRE("1" == flow_file->getAttribute(ConsumeKafka::KAFKA_COUNT_ATTR).value());
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_OFFSET_ATTR));
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_PARTITION_ATTR));
+          REQUIRE(PRODUCER_TOPIC == flow_file->getAttribute(ConsumeKafka::KAFKA_TOPIC_ATTR).value());
+        }
+        flow_files_produced.emplace_back(std::move(flow_file));
+      }
+      plan_->reset_location();
+    }
+
+    const auto contentOrderOfFlowFile = [&] (const std::shared_ptr<core::FlowFile>& lhs, const std::shared_ptr<core::FlowFile>& rhs) {
+      return lhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value() < rhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value();
+    };
+    {
+      SCOPED_INFO("The flowfiles generated by ConsumeKafka are invalid (probably nullptr).");
+      REQUIRE_NOTHROW(std::sort(flow_files_produced.begin(), flow_files_produced.end(), contentOrderOfFlowFile));
+    }
+    std::vector<std::string> sorted_split_messages = sort_and_split_messages(messages_on_topic, message_demarcator);
+    const auto flow_file_content_matches_message = [&] (const std::shared_ptr<core::FlowFile>& flowfile, const std::string message) {
+      return flowfile->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value() == message;
+    };
+
+    logger_->log_debug("************");
+    std::string expected = "Expected: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      expected += sorted_split_messages[i] + ", ";
+    }
+    std::string   actual = "  Actual: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      actual += flow_files_produced[i]->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value() + ", ";
+    }
+    logger_->log_debug("%s", expected.c_str());
+    logger_->log_debug("%s", actual.c_str());
+    logger_->log_debug("************");
+
+    INFO("The messages received by ConsumeKafka do not match those published");
+    REQUIRE(std::equal(flow_files_produced.begin(), flow_files_produced.end(), sorted_split_messages.begin(), flow_file_content_matches_message));
+  }
+};
+
+class ConsumeKafkaContinuousPublishingTest : public ConsumeKafkaTest {
+ public:
+  ConsumeKafkaContinuousPublishingTest() : ConsumeKafkaTest() {}
+  virtual ~ConsumeKafkaContinuousPublishingTest() {
+    logTestController_.reset();
+  }
+
+  void single_consumer_with_continuous_message_producing(
+      const uint64_t msg_periodicity_ms,
+      const std::string& kafka_brokers,
+      const optional<std::string>& group_id,
+      const optional<std::string>& max_poll_records,
+      const optional<std::string>& max_poll_time,
+      const optional<std::string>& session_timeout) {
+    reInitialize();
+
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+
+    plan_->setProperty(consume_kafka, "allow.auto.create.topics", "true", true);  // Seems like the topic tests work without this
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), PRODUCER_TOPIC);
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    consume_kafka->setAutoTerminatedRelationships({success});
+
+    KafkaTestProducer producer("localhost:9092", PRODUCER_TOPIC, /* transactional = */ false);
+
+    std::atomic_bool producer_loop_stop{ false };
+    const auto producer_loop = [&] {
+      std::size_t num_messages_sent = 0;
+      std::this_thread::sleep_for(std::chrono::milliseconds(100));
+      while (!producer_loop_stop) {
+        producer.publish_messages_to_topic({ "Message after " + std::to_string(msg_periodicity_ms * ++num_messages_sent) + " ms"}, TEST_MESSAGE_KEY, { PUBLISH }, {}, {});
+        std::this_thread::sleep_for(std::chrono::milliseconds(msg_periodicity_ms));
+      }
+    };
+
+    plan_->scheduleProcessors();
+
+    const auto get_time_property_ms = [] (const std::string& property_string) {
+      int64_t value;
+      org::apache::nifi::minifi::core::TimeUnit unit;
+      REQUIRE(org::apache::nifi::minifi::core::Property::StringToTime(property_string, value, unit));
+      int64_t value_as_ms;
+      REQUIRE(org::apache::nifi::minifi::core::Property::ConvertTimeUnitToMS(value, unit, value_as_ms));
+      return value_as_ms;
+    };
+
+    std::thread producer_thread(producer_loop);
+    CHECK_NOTHROW(plan_->runNextProcessor());
+    producer_loop_stop = true;
+    producer_thread.join();
+
+    std::size_t num_flow_files_produced = plan_->getNumFlowFileProducedByCurrentProcessor();
+
+    const uint64_t max_poll_time_ms = get_time_property_ms(max_poll_time.value_or(ConsumeKafka::DEFAULT_MAX_POLL_TIME));
+    const uint64_t max_poll_records_value = max_poll_records ? std::stoi(max_poll_records.value()) : ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+    const uint64_t exp_lower_bound = std::min(max_poll_time_ms / msg_periodicity_ms - 2, max_poll_records_value);
+    const uint64_t exp_upper_bound = std::min(max_poll_time_ms / msg_periodicity_ms + 2, max_poll_records_value);
+    logger_->log_debug("Max poll time: %d, Max poll records: %d, Exp. flowfiles produced: (min: %d, max: %d), actual: %d",
+        max_poll_time_ms, max_poll_records_value, exp_lower_bound, exp_upper_bound, num_flow_files_produced);
+
+    REQUIRE(exp_lower_bound <= num_flow_files_produced);
+    REQUIRE(num_flow_files_produced <= exp_upper_bound);
+  }
+};
+
+const std::string ConsumeKafkaTest::TEST_FILE_NAME_POSTFIX{ "target_kafka_message.txt" };
+const std::string ConsumeKafkaTest::TEST_MESSAGE_KEY{ "consume_kafka_test_key" };
+const std::string ConsumeKafkaTest::PRODUCER_TOPIC{ "ConsumeKafkaTest" };
+const std::string ConsumeKafkaTest::ATTRIBUTE_FOR_CAPTURING_CONTENT{ "flowfile_content" };
+const std::chrono::seconds ConsumeKafkaTest::MAX_CONSUMEKAFKA_POLL_TIME_SECONDS{ 5 };
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "ConsumeKafka parses and uses kafka topics.", "[ConsumeKafka][Kafka][Topic]") {
+  auto run_tests = [&] (const std::vector<std::string>& messages_on_topic, const std::string& topic_names, const optional<std::string>& topic_name_format) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", topic_names, topic_name_format, {}, "test_group_id", {}, {}, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({ "Ulysses",              "James Joyce"         }, "ConsumeKafkaTest",         {});
+  run_tests({ "The Great Gatsby",     "F. Scott Fitzgerald" }, "ConsumeKafkaTest",         ConsumeKafka::TOPIC_FORMAT_NAMES);
+  run_tests({ "War and Peace",        "Lev Tolstoy"         }, "a,b,c,ConsumeKafkaTest,d", ConsumeKafka::TOPIC_FORMAT_NAMES);
+  run_tests({ "Nineteen Eighty Four", "George Orwell"       }, "ConsumeKafkaTest",         ConsumeKafka::TOPIC_FORMAT_PATTERNS);
+  run_tests({ "Hamlet",               "William Shakespeare" }, "Cons[emu]*KafkaTest",      ConsumeKafka::TOPIC_FORMAT_PATTERNS);
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Offsets are reset to the latest when a consumer starts with non-processed messages.", "[ConsumeKafka][Kafka][OffsetReset]") {
+  auto run_tests = [&] (
+      const std::vector<std::string>& messages_on_topic,
+      const std::vector<KafkaTestProducer::PublishEvent>& transaction_events) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, transaction_events, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  KafkaTestProducer producer("localhost:9092", PRODUCER_TOPIC, false);
+  producer.publish_messages_to_topic({"Dummy messages", "that should be ignored", "due to offset reset on ConsumeKafka startup"}, TEST_MESSAGE_KEY, {PUBLISH, PUBLISH, PUBLISH}, {}, {});
+  run_tests({"Brave New World",  "Aldous Huxley"}, NON_TRANSACTIONAL_MESSAGES);
+  producer.publish_messages_to_topic({"Dummy messages", "that should be ignored", "due to offset reset on ConsumeKafka startup"}, TEST_MESSAGE_KEY, {PUBLISH, PUBLISH, PUBLISH}, {}, {});
+  run_tests({"Call of the Wild", "Jack London"}, NON_TRANSACTIONAL_MESSAGES);
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Key attribute is encoded according to the \"Key Attribute Encoding\" property.", "[ConsumeKafka][Kafka][KeyAttributeEncoding]") {
+  auto run_tests = [&] (const std::vector<std::string>& messages_on_topic, const optional<std::string>& key_attribute_encoding) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, key_attribute_encoding, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+
+  run_tests({ "The Odyssey",          "Ὅμηρος"                        }, {});
+  run_tests({ "Lolita",               "Владимир Владимирович Набоков" }, "utf-8");
+  run_tests({ "Crime and Punishment", "Фёдор Михайлович Достоевский"  }, "hex");
+  run_tests({ "Paradise Lost",        "John Milton"                   }, "hEX");
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Transactional behaviour is supported.", "[ConsumeKafka][Kafka][Transaction]") {
+  auto run_tests = [&] (const std::vector<std::string>& messages_on_topic, const std::vector<KafkaTestProducer::PublishEvent>& transaction_events, const optional<bool>& honor_transactions) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, transaction_events, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({  "Pride and Prejudice", "Jane Austen"      }, SINGLE_COMMITTED_TRANSACTION, {});
+  run_tests({                 "Dune", "Frank Herbert"    },    TWO_SEPARATE_TRANSACTIONS, {});
+  run_tests({      "The Black Sheep", "Honore De Balzac" },    NON_COMMITTED_TRANSACTION, {});
+  run_tests({     "Gospel of Thomas"                     },        CANCELLED_TRANSACTION, {});
+  run_tests({ "Operation Dark Heart"                     },        CANCELLED_TRANSACTION, true);
+  run_tests({               "Brexit"                     },        CANCELLED_TRANSACTION, false);
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Headers on consumed Kafka messages are extracted into attributes if requested on ConsumeKafka.", "[ConsumeKafka][Kafka][Headers]") {
+  auto run_tests = [&] (
+      const std::vector<std::string>& messages_on_topic,
+      const std::vector<std::pair<std::string, std::string>>& expect_header_attributes,
+      const std::vector<std::pair<std::string, std::string>>& message_headers,
+      const optional<std::string>& headers_to_add_as_attributes,
+      const optional<std::string>& duplicate_header_handling) {
+    single_consumer_with_plain_text_test(true, false, expect_header_attributes, messages_on_topic, NON_TRANSACTIONAL_MESSAGES, message_headers, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, headers_to_add_as_attributes, duplicate_header_handling, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({             "Homeland",   "R. A. Salvatore"},                                      {},             {{{"Contains dark elves"}, {"Yes"}}},         {},                    {});
+  run_tests({             "Magician",  "Raymond E. Feist"},               {{{"Rating"}, {"10/10"}}},                        {{{"Rating"}, {"10/10"}}}, {"Rating"},                    {});
+  run_tests({             "Mistborn", "Brandon Sanderson"},               {{{"Metal"}, {"Copper"}}}, {{{"Metal"}, {"Copper"}}, {{"Metal"}, {"Iron"}}},  {"Metal"},            KEEP_FIRST);
+  run_tests({             "Mistborn", "Brandon Sanderson"},                 {{{"Metal"}, {"Iron"}}}, {{{"Metal"}, {"Copper"}}, {{"Metal"}, {"Iron"}}},  {"Metal"},           KEEP_LATEST);
+  run_tests({             "Mistborn", "Brandon Sanderson"},         {{{"Metal"}, {"Copper, Iron"}}}, {{{"Metal"}, {"Copper"}}, {{"Metal"}, {"Iron"}}},  {"Metal"}, COMMA_SEPARATED_MERGE);
+  run_tests({"The Lord of the Rings",  "J. R. R. Tolkien"}, {{{"Parts"}, {"First, second, third"}}},          {{{"Parts"}, {"First, second, third"}}},  {"Parts"},                    {});
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Messages are separated into multiple flowfiles if the message demarcator is present in the message.", "[ConsumeKafka][Kafka][MessageDemarcator]") {
+  auto run_tests = [&] (
+      const std::vector<std::string>& messages_on_topic,
+      const optional<std::string>& message_demarcator) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, message_demarcator, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({"Barbapapa", "Anette Tison and Talus Taylor"}, "a");
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "The maximum poll records allows ConsumeKafka to combine multiple messages into a single flowfile.", "[ConsumeKafka][Kafka][Batching][MaxPollRecords]") {
+  auto run_tests = [&] (
+      const std::vector<std::string>& messages_on_topic,
+      const std::vector<KafkaTestProducer::PublishEvent>& transaction_events,
+      const optional<std::string>& max_poll_records) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, transaction_events, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, {}, {}, max_poll_records, "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({"The Count of Monte Cristo", "Alexandre Dumas"}, NON_TRANSACTIONAL_MESSAGES, "2");
+
+  const std::vector<std::string> content {
+      "Make const member functions thread safe",
+      "Understand special member function generation",
+      "Use std::unique_ptr for exclusive-ownership resource management",
+      "Use std::shared_ptr for shared-ownership resource management",
+      "Use std::weak_ptr for std::shared_ptr-like pointers that can dangle",
+      "Prefer std::make_unique and std::make_shared to direct use of new",
+      "When using the Pimpl Idiom, define special member functions inthe implementation file",
+      "Understand std::move and std::forward",
+      "Distinguish universal references from rvalue references",
+      "Use std::move on rvalue references, std::forward on universal references",
+      "Avoid overloading on universal references",
+      "Familiarize yourself with alternatives to overloading on universal references",
+      "Understand reference collapsing",
+      "Assume that move operations are not present, not cheap, and not used",
+      "Familiarize yourself with perfect forwarding failure cases",
+      "Avoid default capture modes",
+      "Use init capture to move objects into closures",
+      "Use decltype on auto&& parameters to std::forward them",
+      "Prefer lambdas to std::bind",
+      "Prefer task-based programming to thread-based" };

Review comment:
       Great guidelines, what's the source?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] fgerlits commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
fgerlits commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r576622081



##########
File path: libminifi/test/TestBase.cpp
##########
@@ -61,44 +62,41 @@ TestPlan::~TestPlan() {
   for (auto& processor : configured_processors_) {
     processor->setScheduledState(core::ScheduledState::STOPPED);
   }
+  for (auto& connection : relationships_) {
+    // This is a patch solving circular references between processors and connections
+    connection->setSource(nullptr);
+    connection->setDestination(nullptr);
+  }
   controller_services_provider_->clearControllerServices();
 }
 
 std::shared_ptr<core::Processor> TestPlan::addProcessor(const std::shared_ptr<core::Processor> &processor, const std::string& /*name*/, const std::initializer_list<core::Relationship>& relationships,
-                                                        bool linkToPrevious) {
+    bool linkToPrevious) {
   if (finalized) {
     return nullptr;
   }
   std::lock_guard<std::recursive_mutex> guard(mutex);
-
   utils::Identifier uuid = utils::IdGenerator::getIdGenerator()->generate();
-
   processor->setStreamFactory(stream_factory);
   // initialize the processor
   processor->initialize();
   processor->setFlowIdentifier(flow_version_->getFlowIdentifier());
-
   processor_mapping_[processor->getUUID()] = processor;
-
   if (!linkToPrevious) {
     termination_ = *(relationships.begin());
   } else {
     std::shared_ptr<core::Processor> last = processor_queue_.back();
-
     if (last == nullptr) {
       last = processor;
       termination_ = *(relationships.begin());
     }
-
     std::stringstream connection_name;
     connection_name << last->getUUIDStr() << "-to-" << processor->getUUIDStr();
-    logger_->log_info("Creating %s connection for proc %d", connection_name.str(), processor_queue_.size() + 1);

Review comment:
       I'm not sure how useful this log is, but I suspect you removed it by accident.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r538358865



##########
File path: libminifi/test/TestBase.h
##########
@@ -200,7 +199,7 @@ class LogTestController {
     }
     my_properties_->set("logger.root", "ERROR,ostream");
     my_properties_->set("logger." + core::getClassName<LogTestController>(), "INFO");
-    my_properties_->set("logger." + core::getClassName<logging::LoggerConfiguration>(), "DEBUG");
+    my_properties_->set("logger." + core::getClassName<logging::LoggerConfiguration>(), "INFO");

Review comment:
       This brakes C2JstackTest. Once  #953 is merged, rebasing on it should fix this issue. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553173662



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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() };

Review comment:
       Actually, I am not sure if it is a safe assumption.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r560860773



##########
File path: extensions/librdkafka/tests/ConsumeKafkaTests.cpp
##########
@@ -0,0 +1,590 @@
+
+/**
+ * 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 "../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(), "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 = "Failed to create Kafka producer" + std::string{ errstr.data() };
+      throw std::runtime_error(error_msg);
+    }
+
+    // The last argument is a config here, but it is already owned by the producer. 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> CANCELLED_TRANSACTION        { 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) {
+    reInitialize();
+
+    // Consumer chain
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+    std::shared_ptr<core::Processor> extract_text  = plan_->addProcessor("ExtractText", "extract_text", {success}, false);
+
+    // Set up connections
+    plan_->addConnection(consume_kafka, success, extract_text);
+    extract_text->setAutoTerminatedRelationships({success});
+
+    const auto bool_to_string = [] (const bool b) -> std::string { return b ? "true" : "false"; };
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::SecurityProtocol.getName(), security_protocol);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), topic_names);
+
+    optional_set_property(consume_kafka, ConsumeKafka::TopicNameFormat.getName(), topic_name_format);
+    optional_set_property(consume_kafka, ConsumeKafka::HonorTransactions.getName(), honor_transactions | utils::map(bool_to_string));
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+    optional_set_property(consume_kafka, ConsumeKafka::OffsetReset.getName(), offset_reset);
+    optional_set_property(consume_kafka, ConsumeKafka::KeyAttributeEncoding.getName(), key_attribute_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageDemarcator.getName(), message_demarcator);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageHeaderEncoding.getName(), message_header_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::HeadersToAddAsAttributes.getName(), headers_to_add_as_attributes);
+    optional_set_property(consume_kafka, ConsumeKafka::DuplicateHeaderHandling.getName(), duplicate_header_handling);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    plan_->setProperty(extract_text, ExtractText::Attribute.getName(), ATTRIBUTE_FOR_CAPTURING_CONTENT);
+
+    if (!expect_config_valid) {
+      REQUIRE_THROWS(plan_->scheduleProcessor(consume_kafka));
+      return;
+    } else {
+      plan_->scheduleProcessors();
+    }
+
+    std::unique_ptr<rd_kafka_conf_t, utils::rd_kafka_conf_deleter> conf_;
+    std::unique_ptr<rd_kafka_t, utils::rd_kafka_consumer_deleter> consumer_;
+
+    const bool is_transactional = std::count(transaction_events.cbegin(), transaction_events.cend(), KafkaTestProducer::PublishEvent::TRANSACTION_START);
+    const bool transactions_committed = transaction_events.back() == KafkaTestProducer::PublishEvent::TRANSACTION_COMMIT;
+
+    KafkaTestProducer producer(kafka_brokers, PRODUCER_TOPIC, is_transactional);
+    producer.publish_messages_to_topic(messages_on_topic, TEST_MESSAGE_KEY, transaction_events, message_headers, message_header_encoding);
+
+
+    std::vector<std::shared_ptr<core::FlowFile>> flow_files_produced;
+    for (std::size_t num_expected_messages_processed = 0; num_expected_messages_processed < messages_on_topic.size(); num_expected_messages_processed += std::stoi(max_poll_records.value_or("1"))) {
+      plan_->increment_location();
+      if ((honor_transactions && false == honor_transactions.value()) || (is_transactional && !transactions_committed)) {
+        INFO("Non-committed messages received.");
+        REQUIRE(false == plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+        return;
+      }
+      {
+        SCOPED_INFO("ConsumeKafka timed out when waiting to receive the message published to the kafka broker.");
+        REQUIRE(plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+      }
+      std::size_t num_flow_files_produced = plan_->getNumFlowFileProducedByCurrentProcessor();
+      plan_->increment_location();
+      for (std::size_t times_extract_text_run = 0; times_extract_text_run < num_flow_files_produced; ++times_extract_text_run) {
+        plan_->runCurrentProcessor();  // ExtractText
+        std::shared_ptr<core::FlowFile> flow_file = plan_->getFlowFileProducedByCurrentProcessor();
+        for (const auto& exp_header : expect_header_attributes) {
+          SCOPED_INFO("ConsumeKafka did not produce the expected flowfile attribute from message header: " << exp_header.first << ".");
+          const auto header_attr_opt = flow_file->getAttribute(exp_header.first);
+          REQUIRE(header_attr_opt);
+          REQUIRE(exp_header.second == header_attr_opt.value());
+        }
+        {
+          SCOPED_INFO("Message key is missing or incorrect (potential encoding mismatch).");
+          REQUIRE(TEST_MESSAGE_KEY == decode_key(flow_file->getAttribute(ConsumeKafka::KAFKA_MESSAGE_KEY_ATTR).value(), key_attribute_encoding));
+          REQUIRE("1" == flow_file->getAttribute(ConsumeKafka::KAFKA_COUNT_ATTR).value());
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_OFFSET_ATTR));
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_PARTITION_ATTR));
+          REQUIRE(PRODUCER_TOPIC == flow_file->getAttribute(ConsumeKafka::KAFKA_TOPIC_ATTR).value());
+        }
+        flow_files_produced.emplace_back(std::move(flow_file));
+      }
+      plan_->reset_location();
+    }
+
+    const auto contentOrderOfFlowFile = [&] (const std::shared_ptr<core::FlowFile>& lhs, const std::shared_ptr<core::FlowFile>& rhs) {
+      return lhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value() < rhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value();
+    };
+    {
+      SCOPED_INFO("The flowfiles generated by ConsumeKafka are invalid (probably nullptr).");
+      REQUIRE_NOTHROW(std::sort(flow_files_produced.begin(), flow_files_produced.end(), contentOrderOfFlowFile));
+    }
+    std::vector<std::string> sorted_split_messages = sort_and_split_messages(messages_on_topic, message_demarcator);
+    const auto flow_file_content_matches_message = [&] (const std::shared_ptr<core::FlowFile>& flowfile, const std::string message) {
+      return flowfile->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value() == message;
+    };
+
+    logger_->log_debug("************");
+    std::string expected = "Expected: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      expected += sorted_split_messages[i] + ", ";
+    }
+    std::string   actual = "  Actual: ";
+    for (int i = 0; i < sorted_split_messages.size(); ++i) {
+      actual += flow_files_produced[i]->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value() + ", ";
+    }
+    logger_->log_debug("%s", expected.c_str());
+    logger_->log_debug("%s", actual.c_str());
+    logger_->log_debug("************");
+
+    INFO("The messages received by ConsumeKafka do not match those published");
+    REQUIRE(std::equal(flow_files_produced.begin(), flow_files_produced.end(), sorted_split_messages.begin(), flow_file_content_matches_message));
+  }
+};
+
+class ConsumeKafkaContinuousPublishingTest : public ConsumeKafkaTest {
+ public:
+  ConsumeKafkaContinuousPublishingTest() : ConsumeKafkaTest() {}
+  virtual ~ConsumeKafkaContinuousPublishingTest() {
+    logTestController_.reset();
+  }
+
+  void single_consumer_with_continuous_message_producing(
+      const uint64_t msg_periodicity_ms,
+      const std::string& kafka_brokers,
+      const optional<std::string>& group_id,
+      const optional<std::string>& max_poll_records,
+      const optional<std::string>& max_poll_time,
+      const optional<std::string>& session_timeout) {
+    reInitialize();
+
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+
+    plan_->setProperty(consume_kafka, "allow.auto.create.topics", "true", true);  // Seems like the topic tests work without this
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), PRODUCER_TOPIC);
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    consume_kafka->setAutoTerminatedRelationships({success});
+
+    KafkaTestProducer producer("localhost:9092", PRODUCER_TOPIC, /* transactional = */ false);
+
+    std::atomic_bool producer_loop_stop{ false };
+    const auto producer_loop = [&] {
+      std::size_t num_messages_sent = 0;
+      std::this_thread::sleep_for(std::chrono::milliseconds(100));
+      while (!producer_loop_stop) {
+        producer.publish_messages_to_topic({ "Message after " + std::to_string(msg_periodicity_ms * ++num_messages_sent) + " ms"}, TEST_MESSAGE_KEY, { PUBLISH }, {}, {});
+        std::this_thread::sleep_for(std::chrono::milliseconds(msg_periodicity_ms));
+      }
+    };
+
+    plan_->scheduleProcessors();
+
+    const auto get_time_property_ms = [] (const std::string& property_string) {
+      int64_t value;
+      org::apache::nifi::minifi::core::TimeUnit unit;
+      REQUIRE(org::apache::nifi::minifi::core::Property::StringToTime(property_string, value, unit));
+      int64_t value_as_ms;
+      REQUIRE(org::apache::nifi::minifi::core::Property::ConvertTimeUnitToMS(value, unit, value_as_ms));
+      return value_as_ms;
+    };
+
+    std::thread producer_thread(producer_loop);
+    CHECK_NOTHROW(plan_->runNextProcessor());
+    producer_loop_stop = true;
+    producer_thread.join();
+
+    std::size_t num_flow_files_produced = plan_->getNumFlowFileProducedByCurrentProcessor();
+
+    const uint64_t max_poll_time_ms = get_time_property_ms(max_poll_time.value_or(ConsumeKafka::DEFAULT_MAX_POLL_TIME));
+    const uint64_t max_poll_records_value = max_poll_records ? std::stoi(max_poll_records.value()) : ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+    const uint64_t exp_lower_bound = std::min(max_poll_time_ms / msg_periodicity_ms - 2, max_poll_records_value);
+    const uint64_t exp_upper_bound = std::min(max_poll_time_ms / msg_periodicity_ms + 2, max_poll_records_value);
+    logger_->log_debug("Max poll time: %d, Max poll records: %d, Exp. flowfiles produced: (min: %d, max: %d), actual: %d",
+        max_poll_time_ms, max_poll_records_value, exp_lower_bound, exp_upper_bound, num_flow_files_produced);
+
+    REQUIRE(exp_lower_bound <= num_flow_files_produced);
+    REQUIRE(num_flow_files_produced <= exp_upper_bound);
+  }
+};
+
+const std::string ConsumeKafkaTest::TEST_FILE_NAME_POSTFIX{ "target_kafka_message.txt" };
+const std::string ConsumeKafkaTest::TEST_MESSAGE_KEY{ "consume_kafka_test_key" };
+const std::string ConsumeKafkaTest::PRODUCER_TOPIC{ "ConsumeKafkaTest" };
+const std::string ConsumeKafkaTest::ATTRIBUTE_FOR_CAPTURING_CONTENT{ "flowfile_content" };
+const std::chrono::seconds ConsumeKafkaTest::MAX_CONSUMEKAFKA_POLL_TIME_SECONDS{ 5 };
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "ConsumeKafka parses and uses kafka topics.", "[ConsumeKafka][Kafka][Topic]") {
+  auto run_tests = [&] (const std::vector<std::string>& messages_on_topic, const std::string& topic_names, const optional<std::string>& topic_name_format) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", topic_names, topic_name_format, {}, "test_group_id", {}, {}, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({ "Ulysses",              "James Joyce"         }, "ConsumeKafkaTest",         {});
+  run_tests({ "The Great Gatsby",     "F. Scott Fitzgerald" }, "ConsumeKafkaTest",         ConsumeKafka::TOPIC_FORMAT_NAMES);
+  run_tests({ "War and Peace",        "Lev Tolstoy"         }, "a,b,c,ConsumeKafkaTest,d", ConsumeKafka::TOPIC_FORMAT_NAMES);
+  run_tests({ "Nineteen Eighty Four", "George Orwell"       }, "ConsumeKafkaTest",         ConsumeKafka::TOPIC_FORMAT_PATTERNS);
+  run_tests({ "Hamlet",               "William Shakespeare" }, "Cons[emu]*KafkaTest",      ConsumeKafka::TOPIC_FORMAT_PATTERNS);
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Offsets are reset to the latest when a consumer starts with non-processed messages.", "[ConsumeKafka][Kafka][OffsetReset]") {
+  auto run_tests = [&] (
+      const std::vector<std::string>& messages_on_topic,
+      const std::vector<KafkaTestProducer::PublishEvent>& transaction_events) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, transaction_events, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  KafkaTestProducer producer("localhost:9092", PRODUCER_TOPIC, false);
+  producer.publish_messages_to_topic({"Dummy messages", "that should be ignored", "due to offset reset on ConsumeKafka startup"}, TEST_MESSAGE_KEY, {PUBLISH, PUBLISH, PUBLISH}, {}, {});
+  run_tests({"Brave New World",  "Aldous Huxley"}, NON_TRANSACTIONAL_MESSAGES);
+  producer.publish_messages_to_topic({"Dummy messages", "that should be ignored", "due to offset reset on ConsumeKafka startup"}, TEST_MESSAGE_KEY, {PUBLISH, PUBLISH, PUBLISH}, {}, {});
+  run_tests({"Call of the Wild", "Jack London"}, NON_TRANSACTIONAL_MESSAGES);
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Key attribute is encoded according to the \"Key Attribute Encoding\" property.", "[ConsumeKafka][Kafka][KeyAttributeEncoding]") {
+  auto run_tests = [&] (const std::vector<std::string>& messages_on_topic, const optional<std::string>& key_attribute_encoding) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, key_attribute_encoding, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+
+  run_tests({ "The Odyssey",          "Ὅμηρος"                        }, {});
+  run_tests({ "Lolita",               "Владимир Владимирович Набоков" }, "utf-8");
+  run_tests({ "Crime and Punishment", "Фёдор Михайлович Достоевский"  }, "hex");
+  run_tests({ "Paradise Lost",        "John Milton"                   }, "hEX");
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Transactional behaviour is supported.", "[ConsumeKafka][Kafka][Transaction]") {
+  auto run_tests = [&] (const std::vector<std::string>& messages_on_topic, const std::vector<KafkaTestProducer::PublishEvent>& transaction_events, const optional<bool>& honor_transactions) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, transaction_events, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({  "Pride and Prejudice", "Jane Austen"      }, SINGLE_COMMITTED_TRANSACTION, {});
+  run_tests({                 "Dune", "Frank Herbert"    },    TWO_SEPARATE_TRANSACTIONS, {});
+  run_tests({      "The Black Sheep", "Honore De Balzac" },    NON_COMMITTED_TRANSACTION, {});
+  run_tests({     "Gospel of Thomas"                     },        CANCELLED_TRANSACTION, {});
+  run_tests({ "Operation Dark Heart"                     },        CANCELLED_TRANSACTION, true);
+  run_tests({               "Brexit"                     },        CANCELLED_TRANSACTION, false);
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Headers on consumed Kafka messages are extracted into attributes if requested on ConsumeKafka.", "[ConsumeKafka][Kafka][Headers]") {
+  auto run_tests = [&] (
+      const std::vector<std::string>& messages_on_topic,
+      const std::vector<std::pair<std::string, std::string>>& expect_header_attributes,
+      const std::vector<std::pair<std::string, std::string>>& message_headers,
+      const optional<std::string>& headers_to_add_as_attributes,
+      const optional<std::string>& duplicate_header_handling) {
+    single_consumer_with_plain_text_test(true, false, expect_header_attributes, messages_on_topic, NON_TRANSACTIONAL_MESSAGES, message_headers, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, headers_to_add_as_attributes, duplicate_header_handling, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({             "Homeland",   "R. A. Salvatore"},                                      {},             {{{"Contains dark elves"}, {"Yes"}}},         {},                    {});
+  run_tests({             "Magician",  "Raymond E. Feist"},               {{{"Rating"}, {"10/10"}}},                        {{{"Rating"}, {"10/10"}}}, {"Rating"},                    {});
+  run_tests({             "Mistborn", "Brandon Sanderson"},               {{{"Metal"}, {"Copper"}}}, {{{"Metal"}, {"Copper"}}, {{"Metal"}, {"Iron"}}},  {"Metal"},            KEEP_FIRST);
+  run_tests({             "Mistborn", "Brandon Sanderson"},                 {{{"Metal"}, {"Iron"}}}, {{{"Metal"}, {"Copper"}}, {{"Metal"}, {"Iron"}}},  {"Metal"},           KEEP_LATEST);
+  run_tests({             "Mistborn", "Brandon Sanderson"},         {{{"Metal"}, {"Copper, Iron"}}}, {{{"Metal"}, {"Copper"}}, {{"Metal"}, {"Iron"}}},  {"Metal"}, COMMA_SEPARATED_MERGE);
+  run_tests({"The Lord of the Rings",  "J. R. R. Tolkien"}, {{{"Parts"}, {"First, second, third"}}},          {{{"Parts"}, {"First, second, third"}}},  {"Parts"},                    {});
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Messages are separated into multiple flowfiles if the message demarcator is present in the message.", "[ConsumeKafka][Kafka][MessageDemarcator]") {
+  auto run_tests = [&] (
+      const std::vector<std::string>& messages_on_topic,
+      const optional<std::string>& message_demarcator) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, message_demarcator, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({"Barbapapa", "Anette Tison and Talus Taylor"}, "a");
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "The maximum poll records allows ConsumeKafka to combine multiple messages into a single flowfile.", "[ConsumeKafka][Kafka][Batching][MaxPollRecords]") {
+  auto run_tests = [&] (
+      const std::vector<std::string>& messages_on_topic,
+      const std::vector<KafkaTestProducer::PublishEvent>& transaction_events,
+      const optional<std::string>& max_poll_records) {
+    single_consumer_with_plain_text_test(true, false, {}, messages_on_topic, transaction_events, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, {}, {}, max_poll_records, "2 sec", "60 sec"); // NOLINT
+  };
+  run_tests({"The Count of Monte Cristo", "Alexandre Dumas"}, NON_TRANSACTIONAL_MESSAGES, "2");
+
+  const std::vector<std::string> content {
+      "Make const member functions thread safe",
+      "Understand special member function generation",
+      "Use std::unique_ptr for exclusive-ownership resource management",
+      "Use std::shared_ptr for shared-ownership resource management",
+      "Use std::weak_ptr for std::shared_ptr-like pointers that can dangle",
+      "Prefer std::make_unique and std::make_shared to direct use of new",
+      "When using the Pimpl Idiom, define special member functions inthe implementation file",
+      "Understand std::move and std::forward",
+      "Distinguish universal references from rvalue references",
+      "Use std::move on rvalue references, std::forward on universal references",
+      "Avoid overloading on universal references",
+      "Familiarize yourself with alternatives to overloading on universal references",
+      "Understand reference collapsing",
+      "Assume that move operations are not present, not cheap, and not used",
+      "Familiarize yourself with perfect forwarding failure cases",
+      "Avoid default capture modes",
+      "Use init capture to move objects into closures",
+      "Use decltype on auto&& parameters to std::forward them",
+      "Prefer lambdas to std::bind",
+      "Prefer task-based programming to thread-based" };
+  const std::vector<KafkaTestProducer::PublishEvent> transaction_events(content.size(), PUBLISH);
+  run_tests(content, transaction_events, "5");
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Non-plain text security context throws scheduling exceptions.", "[ConsumeKafka][Kafka][SecurityProtocol]") {
+  single_consumer_with_plain_text_test(false, false, {}, { "Miyamoto Musashi", "Eiji Yoshikawa" }, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", ConsumeKafka::SECURITY_PROTOCOL_SSL, "ConsumeKafkaTest", {}, {}, "test_group_id", {}, {}, {}, {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+}
+
+TEST_CASE_METHOD(ConsumeKafkaPropertiesTest, "Acceptable values for message header and key attribute encoding are \"UTF-8\" and \"hex\".", "[ConsumeKafka][Kafka][InvalidEncoding]") {
+  single_consumer_with_plain_text_test(false, false, {}, {                           "Shogun", "James Clavell" }, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {}, "UTF-32", {},       {}, {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+  single_consumer_with_plain_text_test(false, false, {}, { "Alice's Adventures in Wonderland", "Lewis Carroll" }, NON_TRANSACTIONAL_MESSAGES, {}, "localhost:9092", "PLAINTEXT", "ConsumeKafkaTest", {}, {}, "test_group_id", {},       {}, {}, "UTF-32", {}, {}, "1", "2 sec", "60 sec"); // NOLINT
+}
+
+TEST_CASE_METHOD(ConsumeKafkaContinuousPublishingTest, "ConsumeKafka can spend no more time polling than allowed in the maximum poll time property.", "[ConsumeKafka][Kafka][Batching][MaxPollTime]") {
+  auto run_tests = [&] (
+      const uint64_t msg_periodicity_ms,
+      const optional<std::string>& max_poll_records,
+      const optional<std::string>& max_poll_time,
+      const optional<std::string>& session_timeout) {
+    single_consumer_with_continuous_message_producing(msg_periodicity_ms, "localhost:9092", "test_group_id", max_poll_records, max_poll_time, session_timeout);
+  };
+  // For some reason, a session time-out of a few seconds does not work at all, 10 seconds seems to be stable
+  run_tests(300, "20", "3 seconds", "10000 ms");
+  // Running multiple tests does not work properly here. For some reason, producing messages
+  // while a rebalance is triggered causes this error, and a blocked poll when new
+  // messages are produced:
+  //     Group "test_group_id" heartbeat error response in state up (join state wait-revoke-rebalance_cb, 1 partition(s) assigned): Broker: Group rebalance in progress
+  //
+  //  I tried adding a wait time for more than "session.timeout.ms" inbetween tests, but it was not sufficiend

Review comment:
       Updated.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r538697896



##########
File path: libminifi/include/utils/GeneralUtils.h
##########
@@ -49,6 +49,14 @@ constexpr T intdiv_ceil(T numerator, T denominator) {
       : numerator / denominator + (numerator % denominator != 0));
 }
 
+// from https://stackoverflow.com/questions/15202474
+struct identity {
+    template<typename U>
+    constexpr auto operator()(U&& v) const noexcept -> decltype(std::forward<U>(v)) {
+        return std::forward<U>(v);
+    }
+};
+

Review comment:
       Consider transparently falling back to [`std::identity`](https://en.cppreference.com/w/cpp/utility/functional/identity) when compiled on >=C++20

##########
File path: libminifi/src/utils/StringUtils.cpp
##########
@@ -59,13 +63,21 @@ std::vector<std::string> StringUtils::split(const std::string &str, const std::s
       break;
     }
     auto next = std::find_if(curr, end, is_func);
-    result.push_back(std::string(curr, next));
+    result.push_back(transformation(std::string(curr, next)));

Review comment:
       I would take `transformation` by value. You could also forward it to the call, but this doesn't make sense when it's called more than once, because the move case would call a moved-from function object.
   
   If the caller needs to keep state, they can use `std::ref` and pass by value.

##########
File path: libminifi/test/unit/StringUtilsTests.cpp
##########
@@ -50,6 +50,16 @@ TEST_CASE("TestStringUtils::split4", "[test split classname]") {
   REQUIRE(expected == StringUtils::split(org::apache::nifi::minifi::core::getClassName<org::apache::nifi::minifi::utils::StringUtils>(), "::"));
 }
 
+TEST_CASE("TestStringUtils::split5", "[test split delimiter not specified]") {

Review comment:
       What was the failure? Most languages split between each character on empty delimiter.
   ![2020-12-08-193418_485x47_scrot](https://user-images.githubusercontent.com/1170582/101526035-6c4b2200-398c-11eb-90df-b4bbc57196aa.png)
   




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r534946721



##########
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:
       It was not obvious to me, so I left it commented out and forgotten about it. It is probably completely unneccessary as the testing went fine without it, will remove :)




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553893682



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,569 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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);
+}
+
+namespace {
+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");
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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
+      if (logger -> should_log(core::logging::LOG_LEVEL::info)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));

Review comment:
       I don't mind concatenation when building an error. Introducing a framework dependence here seems overkill, and `join_pack` seems unneccessarily complex. Can I leave it like this?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r577478587



##########
File path: libminifi/test/unit/StringUtilsTests.cpp
##########
@@ -50,6 +50,16 @@ TEST_CASE("TestStringUtils::split4", "[test split classname]") {
   REQUIRE(expected == StringUtils::split(org::apache::nifi::minifi::core::getClassName<org::apache::nifi::minifi::utils::StringUtils>(), "::"));
 }
 
+TEST_CASE("TestStringUtils::split5", "[test split delimiter not specified]") {

Review comment:
       There is no default value for the delimiter. Updated the test description to say `"test split with delimiter set to empty string"`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553891631



##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "rd_kafka configuration error" + error_msg);
+  }
+}
+
+void print_topics_list(std::shared_ptr<logging::Logger> logger, rd_kafka_topic_partition_list_t* kf_topic_partition_list) {
+  for (std::size_t i = 0; i < kf_topic_partition_list->cnt; ++i) {
+    logger->log_debug("kf_topic_partition_list: topic: %s, partition: %d, offset:%lld]",
+    kf_topic_partition_list->elems[i].topic, kf_topic_partition_list->elems[i].partition, kf_topic_partition_list->elems[i].offset);
+  }
+}
+
+void print_kafka_message(const rd_kafka_message_t* rkmessage, const std::shared_ptr<logging::Logger>& logger) {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    const std::string error_msg = "ConsumeKafka: received error message from broker. Librdkafka error msg: " + std::string(rd_kafka_err2str(rkmessage->err));
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, error_msg);
+  }
+  std::string topicName = rd_kafka_topic_name(rkmessage->rkt);
+  std::string message(reinterpret_cast<char*>(rkmessage->payload), rkmessage->len);
+  const char* key = reinterpret_cast<const char*>(rkmessage->key);
+  const std::size_t key_len = rkmessage->key_len;
+  rd_kafka_timestamp_type_t tstype;
+  int64_t timestamp;
+  timestamp = rd_kafka_message_timestamp(rkmessage, &tstype);
+  const char *tsname = "?";
+  if (tstype != RD_KAFKA_TIMESTAMP_NOT_AVAILABLE) {
+    if (tstype == RD_KAFKA_TIMESTAMP_CREATE_TIME) {
+      tsname = "create time";
+    } else if (tstype == RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME) {
+      tsname = "log append time";
+    }
+  }
+  const int64_t seconds_since_timestamp = timestamp ? static_cast<int64_t>(time(NULL)) - static_cast<int64_t>(timestamp / 1000) : 0;
+
+  std::string headers_as_string;
+  rd_kafka_headers_t* hdrs;
+  const rd_kafka_resp_err_t get_header_response = rd_kafka_message_headers(rkmessage, &hdrs);
+  if (RD_KAFKA_RESP_ERR_NO_ERROR == get_header_response) {
+    std::vector<std::string> header_list;
+    kafka_headers_for_each(hdrs, [&] (const std::string& key, const std::string& val) { header_list.emplace_back(key + ": " + val); });
+    headers_as_string = StringUtils::join(", ", header_list);
+  } else if (RD_KAFKA_RESP_ERR__NOENT != 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::string message_as_string;
+  message_as_string += "[Topic](" + topicName + "), ";
+  message_as_string += "[Key](" + (key != nullptr ? std::string(key, key_len) : std::string("[None]")) + "), ";
+  message_as_string += "[Offset](" +  std::to_string(rkmessage->offset) + "), ";
+  message_as_string += "[Message Length](" + std::to_string(rkmessage->len) + "), ";
+  message_as_string += "[Timestamp](" + std::string(tsname) + " " + std::to_string(timestamp) + " (" + std::to_string(seconds_since_timestamp) + " s ago)), ";
+  message_as_string += "[Headers](";
+  message_as_string += headers_as_string + "\n";
+  message_as_string += "[Payload](" + message + ")";
+
+  logger -> log_debug("Message: %s", message_as_string.c_str());
+}
+
+std::string get_encoded_string(const std::string& input, KafkaEncoding encoding) {
+  switch (encoding) {
+    case KafkaEncoding::UTF8:
+      return input;
+    case KafkaEncoding::HEX:
+      return StringUtils::to_hex(input, /* uppercase = */ true);
+  }
+  throw std::runtime_error("Invalid encoding selected for encoding.");

Review comment:
       Updated but with simple string concatenation, I usually refrain from using fmt unless there is a practical need.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553324753



##########
File path: extensions/librdkafka/tests/ConsumeKafkaTests.cpp
##########
@@ -0,0 +1,593 @@
+
+/**
+ * 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 "../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(), "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) {
+    reInitialize();
+
+    // Consumer chain
+    std::shared_ptr<core::Processor> consume_kafka = plan_->addProcessor("ConsumeKafka", "consume_kafka", {success}, false);
+    std::shared_ptr<core::Processor> extract_text  = plan_->addProcessor("ExtractText", "extract_text", {success}, false);
+
+    // Set up connections
+    plan_->addConnection(consume_kafka, success, extract_text);
+    extract_text->setAutoTerminatedRelationships({success});
+
+    const auto bool_to_string = [] (const bool b) -> std::string { return b ? "true" : "false"; };
+
+    plan_->setProperty(consume_kafka, ConsumeKafka::KafkaBrokers.getName(), kafka_brokers);
+    plan_->setProperty(consume_kafka, ConsumeKafka::SecurityProtocol.getName(), security_protocol);
+    plan_->setProperty(consume_kafka, ConsumeKafka::TopicNames.getName(), topic_names);
+
+    optional_set_property(consume_kafka, ConsumeKafka::TopicNameFormat.getName(), topic_name_format);
+    optional_set_property(consume_kafka, ConsumeKafka::HonorTransactions.getName(), honor_transactions | utils::map(bool_to_string));
+    optional_set_property(consume_kafka, ConsumeKafka::GroupID.getName(), group_id);
+    optional_set_property(consume_kafka, ConsumeKafka::OffsetReset.getName(), offset_reset);
+    optional_set_property(consume_kafka, ConsumeKafka::KeyAttributeEncoding.getName(), key_attribute_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageDemarcator.getName(), message_demarcator);
+    optional_set_property(consume_kafka, ConsumeKafka::MessageHeaderEncoding.getName(), message_header_encoding);
+    optional_set_property(consume_kafka, ConsumeKafka::HeadersToAddAsAttributes.getName(), headers_to_add_as_attributes);
+    optional_set_property(consume_kafka, ConsumeKafka::DuplicateHeaderHandling.getName(), duplicate_header_handling);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollRecords.getName(), max_poll_records);
+    optional_set_property(consume_kafka, ConsumeKafka::MaxPollTime.getName(), max_poll_time);
+    optional_set_property(consume_kafka, ConsumeKafka::SessionTimeout.getName(), session_timeout);
+
+    plan_->setProperty(extract_text, ExtractText::Attribute.getName(), ATTRIBUTE_FOR_CAPTURING_CONTENT);
+
+    if (!expect_config_valid) {
+      const auto& message = messages_on_topic.front();
+      REQUIRE_THROWS(plan_->scheduleProcessor(consume_kafka));
+      return;
+    } else {
+      plan_->scheduleProcessors();
+    }
+
+    std::unique_ptr<rd_kafka_conf_t, utils::rd_kafka_conf_deleter> conf_;
+    std::unique_ptr<rd_kafka_t, utils::rd_kafka_consumer_deleter> consumer_;
+
+    const bool is_transactional = std::count(transaction_events.cbegin(), transaction_events.cend(), KafkaTestProducer::PublishEvent::TRANSACTION_START);
+    const bool transactions_committed = transaction_events.back() == KafkaTestProducer::PublishEvent::TRANSACTION_COMMIT;
+
+    KafkaTestProducer producer(kafka_brokers, PRODUCER_TOPIC, is_transactional);
+    producer.publish_messages_to_topic(messages_on_topic, TEST_MESSAGE_KEY, transaction_events, message_headers, message_header_encoding);
+
+
+    std::vector<std::shared_ptr<core::FlowFile>> flow_files_produced;
+    for (std::size_t num_expected_messages_processed = 0; num_expected_messages_processed < messages_on_topic.size(); num_expected_messages_processed += std::stoi(max_poll_records.value_or("1"))) {
+      plan_->increment_location();
+      if ((honor_transactions && false == honor_transactions.value()) || (is_transactional && !transactions_committed)) {
+        INFO("Non-committed messages received.");
+        REQUIRE(false == plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+        return;
+      }
+      {
+        SCOPED_INFO("ConsumeKafka timed out when waiting to receive the message published to the kafka broker.");
+        REQUIRE(plan_->runCurrentProcessorUntilFlowfileIsProduced(MAX_CONSUMEKAFKA_POLL_TIME_SECONDS));
+      }
+      std::size_t num_flow_files_produced = plan_->getNumFlowFileProducedByCurrentProcessor();
+      plan_->increment_location();
+      for (std::size_t times_extract_text_run = 0; times_extract_text_run < num_flow_files_produced; ++times_extract_text_run) {
+        plan_->runCurrentProcessor();  // ExtractText
+        std::shared_ptr<core::FlowFile> flow_file = plan_->getFlowFileProducedByCurrentProcessor();
+        for (const auto& exp_header : expect_header_attributes) {
+          SCOPED_INFO("ConsumeKafka did not produce the expected flowfile attribute from message header: " << exp_header.first << ".");
+          const auto header_attr_opt = flow_file->getAttribute(exp_header.first);
+          REQUIRE(header_attr_opt);
+          REQUIRE(exp_header.second == header_attr_opt.value().get());
+        }
+        {
+          SCOPED_INFO("Message key is missing or incorrect (potential encoding mismatch).");
+          REQUIRE(TEST_MESSAGE_KEY == decode_key(flow_file->getAttribute(ConsumeKafka::KAFKA_MESSAGE_KEY_ATTR).value().get(), key_attribute_encoding));
+          REQUIRE("1" == flow_file->getAttribute(ConsumeKafka::KAFKA_COUNT_ATTR).value().get());
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_OFFSET_ATTR));
+          REQUIRE(flow_file->getAttribute(ConsumeKafka::KAFKA_PARTITION_ATTR));
+          REQUIRE(PRODUCER_TOPIC == flow_file->getAttribute(ConsumeKafka::KAFKA_TOPIC_ATTR).value().get());
+        }
+        flow_files_produced.emplace_back(std::move(flow_file));
+      }
+      plan_->reset_location();
+    }
+
+    const auto contentOrderOfFlowFile = [&] (const std::shared_ptr<core::FlowFile>& lhs, const std::shared_ptr<core::FlowFile>& rhs) {
+      return lhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get() < rhs->getAttribute(ATTRIBUTE_FOR_CAPTURING_CONTENT).value().get();
+    };
+    {
+      SCOPED_INFO("The flowfiles generated by ConsumeKafka are invalid (probably nullptr).");
+      CHECK_NOTHROW(std::sort(flow_files_produced.begin(), flow_files_produced.end(), contentOrderOfFlowFile));

Review comment:
       I am not sure `REQUIRE_NOTHROW` supports scoped info. Updated nonetheless, if it fails here, it should be easy to see why.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553914086



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,582 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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());

Review comment:
       Added bolding for `Topic Names`, `Topic Name Format` and `Group ID`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r577502720



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,578 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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 comma separated list to match against all message headers. Any message header whose name matches an item from the list will be added to the FlowFile "
+      "as an Attribute. If not specified, no Header values will be added as FlowFile attributes. The behaviour on when multiple headers of the same name are present is set using "
+      "the DuplicateHeaderHandling attribute.")
+  ->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 */) {
+  gsl_Expects(context);
+  // 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);
+}
+
+namespace {
+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");
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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));
+}
+}  // namespace
+
+void ConsumeKafka::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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.data() };
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Failed to create Kafka consumer " + error_msg);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  if (RD_KAFKA_RESP_ERR_NO_ERROR != rkmessage->err) {
+    throw minifi::Exception(ExceptionType::PROCESSOR_EXCEPTION, "ConsumeKafka: received error message from broker: " + std::to_string(rkmessage->err) + " " + rd_kafka_err2str(rkmessage->err));
+  }
+  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::steady_clock::now();
+  auto elapsed = std::chrono::steady_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());
+    std::unique_ptr<rd_kafka_message_t, utils::rd_kafka_message_deleter>
+      message { rd_kafka_consumer_poll(consumer_.get(), std::chrono::duration_cast<std::chrono::milliseconds>(max_poll_time_milliseconds_ - elapsed).count()), utils::rd_kafka_message_deleter() };
+    if (!message) {
+      break;
+    }
+    if (RD_KAFKA_RESP_ERR_NO_ERROR != message->err) {
+      logger_->log_error("Received message with error %d: %s", message->err, rd_kafka_err2str(message->err));
+      break;
+    }
+    utils::print_kafka_message(message.get(), *logger_);
+    messages.emplace_back(std::move(message));
+    elapsed = std::chrono::steady_clock::now() - start;
+  }
+  return messages;
+}
+
+utils::KafkaEncoding ConsumeKafka::key_attr_encoding_attr_to_enum() const {
+  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() const {
+  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, "\"Message Header Encoding\" property not recognized.");
+}
+
+std::string ConsumeKafka::resolve_duplicate_headers(const std::vector<std::string>& matching_headers) const {
+  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) const {
+  // 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 < 200) {
+      logger_->log_debug("%.*s", static_cast<int>(size), value);
+    } else {
+      logger_->log_debug("%.*s...", 200, 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) const {
+  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_) {
+      ret = stream->write(data_,  gsl::narrow<int>(dataSize_));
+    }
+    return ret;
+  }
+};

Review comment:
       Moved this to a proper location inside ConsumeKafka's class scope.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r577493327



##########
File path: extensions/librdkafka/ConsumeKafka.h
##########
@@ -0,0 +1,181 @@
+/**
+ * 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 <memory>
+#include <string>
+#include <utility>
+#include <vector>
+
+#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 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()) :

Review comment:
       I wouldn't bother, but feel free if you like.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r528642934



##########
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();
+  }
+};

Review comment:
       This is probably not the proper place for a validator. However I do not know where it should go.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] fgerlits commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
fgerlits commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553422538



##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -0,0 +1,115 @@
+/**
+ * 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 <array>
+
+#include "rdkafka_utils.h"
+
+#include "Exception.h"
+#include "utils/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void setKafkaConfigurationField(rd_kafka_conf_t* configuration, const std::string& field_name, const std::string& value) {
+  static std::array<char, 512U> errstr{};
+  rd_kafka_conf_res_t result;
+  result = rd_kafka_conf_set(configuration, field_name.c_str(), value.c_str(), errstr.data(), errstr.size());
+  if (RD_KAFKA_CONF_OK != result) {
+    const std::string error_msg { errstr.begin(), errstr.end() };

Review comment:
       This looks resolved now.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553890229



##########
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;
+    // }
+    rd_kafka_destroy(ptr);
+  }
+};
+
+struct rd_kafka_consumer_deleter {
+  void operator()(rd_kafka_t* ptr) const noexcept {
+    rd_kafka_consumer_close(ptr);
+    rd_kafka_destroy(ptr);
+  }
+};
+
+struct rd_kafka_topic_partition_list_deleter {
+  void operator()(rd_kafka_topic_partition_list_t* ptr) const noexcept { rd_kafka_topic_partition_list_destroy(ptr); }
+};
+
+struct rd_kafka_topic_conf_deleter {
+  void operator()(rd_kafka_topic_conf_t* ptr) const noexcept { rd_kafka_topic_conf_destroy(ptr); }
+};
+struct rd_kafka_topic_deleter {
+  void operator()(rd_kafka_topic_t* ptr) const noexcept { rd_kafka_topic_destroy(ptr); }
+};
+
+struct rd_kafka_message_deleter {
+  void operator()(rd_kafka_message_t* ptr) const noexcept { rd_kafka_message_destroy(ptr); }
+};
+
+struct rd_kafka_headers_deleter {
+  void operator()(rd_kafka_headers_t* ptr) const noexcept { rd_kafka_headers_destroy(ptr); }
+};
+
+template <typename T>
+void kafka_headers_for_each(const rd_kafka_headers_t* headers, T&& key_value_handle) {
+  const char *key;  // Null terminated, not to be freed
+  const void *value;
+  std::size_t size;
+  for (std::size_t i = 0; RD_KAFKA_RESP_ERR_NO_ERROR == rd_kafka_header_get_all(headers, i, &key, &value, &size); ++i) {
+    std::forward<T>(key_value_handle)(std::string(key), std::string(static_cast<const char*>(value), size));

Review comment:
       Good call! Updated.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r528648823



##########
File path: libminifi/test/TestBase.cpp
##########
@@ -61,78 +62,51 @@ TestPlan::~TestPlan() {
   for (auto& processor : configured_processors_) {
     processor->setScheduledState(core::ScheduledState::STOPPED);
   }
+  for (auto& connection : relationships_) {

Review comment:
       There was an issue where we could not get processors to stop due to them being cross-referenced by their corresponding connections, holding cross-ownerships. This prevented them to be properly freed in the testcases using `TestBase`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553187713



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  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();

Review comment:
       Replaced.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553871478



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  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() const {
+  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() const {
+  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) const {
+  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) const {
+  // 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()) {

Review comment:
       Added a 200 character limit on the logs.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r534270988



##########
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:
       Can you please ellaborate?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553197371



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  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() const {
+  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() const {
+  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) const {
+  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) const {
+  // 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()) {

Review comment:
       I am not sure if there is something inherently wrong with having long messages, except that we cannot reliably log them. It *might* be a use-case.

##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);
+  }
+
+  create_topic_partition_list();
+
+  // 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) const {
+  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() const {
+  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() const {
+  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.");

Review comment:
       Updated exception text.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553169167



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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);

Review comment:
       thanks




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r553216382



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,553 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->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 MiNiFi 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, MiNiFi 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::create_topic_partition_list() {
+  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 (utils::StringUtils::equalsIgnoreCase(TOPIC_FORMAT_PATTERNS, topic_name_format_)) {
+    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() };

Review comment:
       I'm pretty sure it's null-terminated, because C strings are null-terminated and because the API has no way of returning the number of characters written to the error buffer.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #940: MINIFICPP-1373 - Implement ConsumeKafka

Posted by GitBox <gi...@apache.org>.
hunyadi-dev commented on a change in pull request #940:
URL: https://github.com/apache/nifi-minifi-cpp/pull/940#discussion_r577493724



##########
File path: extensions/librdkafka/ConsumeKafka.cpp
##########
@@ -0,0 +1,578 @@
+/**
+ * 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 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 {
+
+constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
+constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
+
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
+constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
+
+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. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(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)
+  ->isRequired(true)
+  ->build());
+
+core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi 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, MiNiFi 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)
+  ->isRequired(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 comma separated list to match against all message headers. Any message header whose name matches an item from the list will be added to the FlowFile "
+      "as an Attribute. If not specified, no Header values will be added as FlowFile attributes. The behaviour on when multiple headers of the same name are present is set using "
+      "the DuplicateHeaderHandling attribute.")
+  ->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 */) {
+  gsl_Expects(context);
+  // 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);
+}
+
+namespace {
+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");
+      if (logger->should_log(core::logging::LOG_LEVEL::debug)) {
+        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:");

Review comment:
       Added missing colon to print `"assigned:"`.




----------------------------------------------------------------
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:
users@infra.apache.org