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 2021/02/01 09:29:18 UTC

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

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