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

2021-02-18 Thread GitBox


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 
+#include 
+#include 
+#include 
+#include 
+#include 
+#include 
+
+#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, 1 /* 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 
+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 
+
+#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 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].pa

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

2021-02-18 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::Propert

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

2021-02-17 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::Propert

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

2021-02-17 Thread GitBox


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




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

2021-02-17 Thread GitBox


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 
+#include 
+#include 
+#include 
+
+#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{ 1 };
+  static constexpr char const* DEFAULT_MAX_POLL_TIME = "4 seconds";
+  static constexpr const std::size_t METADATA_COMMUNICATIONS_TIMEOUT_MS{ 6 
};
+
+  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:
us...@infra.apache.org




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

2021-02-17 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::Propert

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

2021-02-17 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::Propert

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

2021-02-17 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::Propert

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

2021-02-17 Thread GitBox


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 
+#include 
+#include 
+#include 
+
+#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{ 1 };
+  static constexpr char const* DEFAULT_MAX_POLL_TIME = "4 seconds";
+  static constexpr const std::size_t METADATA_COMMUNICATIONS_TIMEOUT_MS{ 6 
};
+
+  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:
us...@infra.apache.org




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

2021-02-17 Thread GitBox


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 TestPlan::addProcessor(const 
std::shared_ptr &processor, const std::string& /*name*/, const 
std::initializer_list& relationships,
-bool linkToPrevious) {
+bool linkToPrevious) {
   if (finalized) {
 return nullptr;
   }
   std::lock_guard 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 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:
us...@infra.apache.org




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

2021-02-17 Thread GitBox


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(),
 "::"));
 }
 
+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:
us...@infra.apache.org




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

2021-02-17 Thread GitBox


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 
TestPlan::addProcessor(const std::shared_ptr node = 
std::make_shared(processor);
-
   processor_nodes_.push_back(node);
-
   // std::shared_ptr context = 
std::make_shared(node, controller_services_provider_, 
prov_repo_, flow_repo_, configuration_, content_repo_);
-
   auto contextBuilder = 
core::ClassLoader::getDefaultClassLoader().instantiate("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 TestPlan::addProcessor(const std::string 
&processor_name, const utils::Identifier& uuid, const std::string &name,
-const 
std::initializer_list& relationships, bool linkToPrevious) {
+  const std::initializer_list& 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:
us...@infra.apache.org




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

2021-02-17 Thread GitBox


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 
TestPlan::addProcessor(const std::string &proce
 }
 
 std::shared_ptr TestPlan::addProcessor(const std::string 
&processor_name, const std::string &name, const 
std::initializer_list& 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:
us...@infra.apache.org




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

2021-02-15 Thread GitBox


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 
+#include 
+#include 
+#include 
+#include 
+#include 
+#include 
+
+#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, 1 /* 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 
+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(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:
us...@infra.apache.org




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

2021-02-15 Thread GitBox


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 TestPlan::addProcessor(const 
std::shared_ptr &processor, const std::string &name, const 
std::initializer_list& relationships,
-bool linkToPrevious) {
+bool linkToPrevious) {
   if (finalized) {
 return nullptr;
   }
   std::lock_guard 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 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 connection = 
std::make_shared(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:
us...@infra.apache.org




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

2021-02-01 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::Propert

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

2021-02-01 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::Propert

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

2021-02-01 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::Propert

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

2021-02-01 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::Propert

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

2021-02-01 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::Propert

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

2021-02-01 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::Propert

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

2021-02-01 Thread GitBox


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 TestPlan::addProcessor(const 
std::shared_ptr &processor, const std::string &name, const 
std::initializer_list& relationships,
-bool linkToPrevious) {
+bool linkToPrevious) {
   if (finalized) {
 return nullptr;
   }
   std::lock_guard 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 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 connection = 
std::make_shared(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:
us...@infra.apache.org




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

2021-01-20 Thread GitBox


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 
+#include 
+#include 
+#include 
+
+#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::getLogger()) {
+using utils::setKafkaConfigurationField;
+
+std::unique_ptr 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 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& messages_on_topic, const std::string& 
message_key, std::vector events,
+  const std::vector>& message_headers, 
const optional& 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>& message_headers, const 
optional& message_header_encoding) {
+logger_->log_debug("Producing: %s", message.c_str());
+std::unique_ptr 
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& message_header : 
message_headers) {
+  rd_kafka_header_add(headers.get(),
+  const_cast(message_header.first.c_s

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

2021-01-20 Thread GitBox


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 
+
+#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 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:
us...@infra.apache.org




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

2021-01-20 Thread GitBox


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 
+#include 
+#include 
+#include 
+
+#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::getLogger()) {
+using utils::setKafkaConfigurationField;
+
+std::unique_ptr 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 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& messages_on_topic, const std::string& 
message_key, std::vector events,
+  const std::vector>& message_headers, 
const optional& 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>& message_headers, const 
optional& message_header_encoding) {
+logger_->log_debug("Producing: %s", message.c_str());
+std::unique_ptr 
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& message_header : 
message_headers) {
+  rd_kafka_header_add(headers.get(),
+  const_cast(message_header.first.c_s

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

2021-01-20 Thread GitBox


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 
+#include 
+#include 
+#include 
+#include 
+#include 
+#include 
+
+#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, 1 /* 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 
+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(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:
us...@infra.apache.org




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

2021-01-20 Thread GitBox


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 
+
+#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 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(rkmessage->payload), 
rkmessage->len);
+  const char* key = reinterpret_cast(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 
+
+#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 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::si

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

2021-01-20 Thread GitBox


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 
+
+#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 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:
us...@infra.apache.org




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

2021-01-20 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::Propert

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

2021-01-08 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::Propert

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

2021-01-08 Thread GitBox


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




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

2021-01-08 Thread GitBox


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 
+#include 
+#include 
+#include 
+
+#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::getLogger()) {
+using utils::setKafkaConfigurationField;
+
+std::unique_ptr 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 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& messages_on_topic, const std::string& 
message_key, std::vector events,
+  const std::vector>& message_headers, 
const optional& 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>& message_headers, const 
optional& message_header_encoding) {
+logger_->log_debug("Producing: %s", message.c_str());
+std::unique_ptr 
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& message_header : 
message_headers) {
+  rd_kafka_header_add(headers.get(),
+  const_cast(message_header.first.c_s

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

2021-01-08 Thread GitBox


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




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

2021-01-08 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::Propert

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

2021-01-08 Thread GitBox


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 
+
+#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 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 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& 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(rkmessage->payload), 
rkmessage->len);
+  const char* key = reinterpret_cast(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(time(NULL)) - static_cast(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:
us...@infra.apache.org




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

2021-01-08 Thread GitBox


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 
+
+#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 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 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& 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(rkmessage->payload), 
rkmessage->len);
+  const char* key = reinterpret_cast(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(time(NULL)) - static_cast(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:
us...@infra.apache.org




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

2021-01-08 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::Propert

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

2021-01-08 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::Propert

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

2021-01-08 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::Propert

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

2021-01-08 Thread GitBox


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 
+
+#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 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 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& 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:
us...@infra.apache.org




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

2021-01-08 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::Propert

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

2021-01-08 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::Propert

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

2021-01-08 Thread GitBox


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 
+
+#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 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 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& 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(rkmessage->payload), 
rkmessage->len);
+  const char* key = reinterpret_cast(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(time(NULL)) - static_cast(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 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);
+  }
+ 

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

2021-01-08 Thread GitBox


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 
+
+#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 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:
us...@infra.apache.org




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

2021-01-08 Thread GitBox


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 
+
+#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 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 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& 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(rkmessage->payload), 
rkmessage->len);
+  const char* key = reinterpret_cast(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(time(NULL)) - static_cast(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 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 

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

2021-01-08 Thread GitBox


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 
+#include 
+#include 
+#include 
+#include 
+#include 
+#include 
+
+#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, 1 /* 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 
+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(key_value_handle)(std::string(key), 
std::string(static_cast(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:
us...@infra.apache.org




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

2021-01-08 Thread GitBox


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 TestPlan::addProcessor(const 
std::shared_ptr &processor, const std::string &name, const 
std::initializer_list& relationships,
-bool linkToPrevious) {
+bool linkToPrevious) {
   if (finalized) {
 return nullptr;
   }
   std::lock_guard 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 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 connection = 
std::make_shared(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 TestPlan::addProcessor(const 
std::shared_ptr &processor, const std::string &name, const 
std::initializer_list& relationships,
-bool linkToPrevious) {
+bool linkToPrevious) {
   if (finalized) {
 return nullptr;
   }
   std::lock_guard 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 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 connection = 
std::make_shared(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 con

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

2021-01-08 Thread GitBox


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 
+#include 
+
+#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 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 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 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:
us...@infra.apache.org




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

2021-01-08 Thread GitBox


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




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

2021-01-08 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::PropertyBuilder::createProperty("Group ID")
+  ->wi

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

2021-01-08 Thread GitBox


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 
+#include 
+
+#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 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 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 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 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:
us...@infra.apache.org




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

2021-01-08 Thread GitBox


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 
+#include 
+
+#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 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 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 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:
us...@infra.apache.org




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

2021-01-07 Thread GitBox


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 
+#include 
+
+#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 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 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 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:
us...@infra.apache.org




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

2021-01-07 Thread GitBox


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 StringUtils::split(const std::string &str, const 
std::string &delimiter) {
+template
+std::vector split_transformed(const std::string& str, const 
std::string& delimiter, Fun transformation) {
   std::vector 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 StringUtils::split(const std::string &str, const 
std::string &delimiter) {
+template
+std::vector split_transformed(const std::string& str, const 
std::string& delimiter, Fun transformation) {
   std::vector 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:
us...@infra.apache.org




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

2021-01-07 Thread GitBox


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(), 
"INFO");
-my_properties_->set("logger." + 
core::getClassName(), "DEBUG");
+my_properties_->set("logger." + 
core::getClassName(), "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(), 
"INFO");
-my_properties_->set("logger." + 
core::getClassName(), "DEBUG");
+my_properties_->set("logger." + 
core::getClassName(), "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(), 
"INFO");
-my_properties_->set("logger." + 
core::getClassName(), "DEBUG");
+my_properties_->set("logger." + 
core::getClassName(), "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:
us...@infra.apache.org




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

2021-01-07 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::Propert

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

2021-01-07 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::Propert

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

2021-01-07 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::Propert

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

2021-01-07 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::PropertyBuilder::createProperty("Group ID")
+  ->wi

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

2021-01-07 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::Propert

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

2021-01-07 Thread GitBox


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(), 
"INFO");
-my_properties_->set("logger." + 
core::getClassName(), "DEBUG");
+my_properties_->set("logger." + 
core::getClassName(), "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:
us...@infra.apache.org




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

2021-01-07 Thread GitBox


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




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

2021-01-07 Thread GitBox


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 
+#include 
+#include 
+#include 
+
+#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::getLogger()) {
+using utils::setKafkaConfigurationField;
+
+std::unique_ptr 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 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& messages_on_topic, const std::string& 
message_key, std::vector events,
+  const std::vector>& message_headers, 
const optional& 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>& message_headers, const 
optional& message_header_encoding) {
+logger_->log_debug("Producing: %s", message.c_str());
+std::unique_ptr 
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& message_header : 
message_headers) {
+  rd_kafka_header_add(headers.get(),
+  const_cast(messag

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

2021-01-07 Thread GitBox


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 
+#include 
+#include 
+#include 
+
+#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::getLogger()) {
+using utils::setKafkaConfigurationField;
+
+std::unique_ptr 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 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& messages_on_topic, const std::string& 
message_key, std::vector events,
+  const std::vector>& message_headers, 
const optional& 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>& message_headers, const 
optional& message_header_encoding) {
+logger_->log_debug("Producing: %s", message.c_str());
+std::unique_ptr 
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& message_header : 
message_headers) {
+  rd_kafka_header_add(headers.get(),
+  const_cast(messag

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

2021-01-07 Thread GitBox


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 
+#include 
+#include 
+#include 
+
+#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::getLogger()) {
+using utils::setKafkaConfigurationField;
+
+std::unique_ptr 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 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& messages_on_topic, const std::string& 
message_key, std::vector events,
+  const std::vector>& message_headers, 
const optional& 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>& message_headers, const 
optional& message_header_encoding) {
+logger_->log_debug("Producing: %s", message.c_str());
+std::unique_ptr 
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& message_header : 
message_headers) {
+  rd_kafka_header_add(headers.get(),
+  const_cast(messag

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

2021-01-07 Thread GitBox


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 
+#include 
+#include 
+#include 
+
+#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::getLogger()) {
+using utils::setKafkaConfigurationField;
+
+std::unique_ptr 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 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& messages_on_topic, const std::string& 
message_key, std::vector events,
+  const std::vector>& message_headers, 
const optional& 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>& message_headers, const 
optional& message_header_encoding) {
+logger_->log_debug("Producing: %s", message.c_str());
+std::unique_ptr 
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& message_header : 
message_headers) {
+  rd_kafka_header_add(headers.get(),
+  const_cast(messag

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

2021-01-07 Thread GitBox


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 
+#include 
+#include 
+#include 
+
+#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::getLogger()) {
+using utils::setKafkaConfigurationField;
+
+std::unique_ptr 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 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& messages_on_topic, const std::string& 
message_key, std::vector events,
+  const std::vector>& message_headers, 
const optional& 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>& message_headers, const 
optional& message_header_encoding) {
+logger_->log_debug("Producing: %s", message.c_str());
+std::unique_ptr 
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& message_header : 
message_headers) {
+  rd_kafka_header_add(headers.get(),
+  const_cast(messag

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

2021-01-07 Thread GitBox


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 
+#include 
+#include 
+#include 
+
+#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::getLogger()) {
+using utils::setKafkaConfigurationField;
+
+std::unique_ptr 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 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& messages_on_topic, const std::string& 
message_key, std::vector events,
+  const std::vector>& message_headers, 
const optional& 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>& message_headers, const 
optional& message_header_encoding) {
+logger_->log_debug("Producing: %s", message.c_str());
+std::unique_ptr 
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& message_header : 
message_headers) {
+  rd_kafka_header_add(headers.get(),
+  const_cast(messag

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

2021-01-07 Thread GitBox


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 
+#include 
+#include 
+#include 
+
+#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::getLogger()) {
+using utils::setKafkaConfigurationField;
+
+std::unique_ptr 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 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& messages_on_topic, const std::string& 
message_key, std::vector events,
+  const std::vector>& message_headers, 
const optional& 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>& message_headers, const 
optional& message_header_encoding) {
+logger_->log_debug("Producing: %s", message.c_str());
+std::unique_ptr 
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& message_header : 
message_headers) {
+  rd_kafka_header_add(headers.get(),
+  const_cast(messag

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

2021-01-07 Thread GitBox


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 
+#include 
+#include 
+#include 
+
+#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::getLogger()) {
+using utils::setKafkaConfigurationField;
+
+std::unique_ptr 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 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& messages_on_topic, const std::string& 
message_key, std::vector events,
+  const std::vector>& message_headers, 
const optional& 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>& message_headers, const 
optional& message_header_encoding) {
+logger_->log_debug("Producing: %s", message.c_str());
+std::unique_ptr 
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& message_header : 
message_headers) {
+  rd_kafka_header_add(headers.get(),
+  const_cast(messag

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

2021-01-07 Thread GitBox


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 
+#include 
+#include 
+#include 
+
+#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::getLogger()) {
+using utils::setKafkaConfigurationField;
+
+std::unique_ptr 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 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:
us...@infra.apache.org




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

2021-01-07 Thread GitBox


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 
+#include 
+#include 
+#include 
+
+#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::getLogger()) {
+using utils::setKafkaConfigurationField;
+
+std::unique_ptr 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 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:
us...@infra.apache.org




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

2021-01-07 Thread GitBox


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 
+
+#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 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 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& 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(rkmessage->payload), 
rkmessage->len);
+  const char* key = reinterpret_cast(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(time(NULL)) - static_cast(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 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:
us...@infra.apache.org




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

2021-01-07 Thread GitBox


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 
+
+#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 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 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& 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(rkmessage->payload), 
rkmessage->len);
+  const char* key = reinterpret_cast(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(time(NULL)) - static_cast(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 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:
us...@infra.apache.org




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

2021-01-07 Thread GitBox


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 
+
+#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 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 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& 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(rkmessage->payload), 
rkmessage->len);
+  const char* key = reinterpret_cast(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(time(NULL)) - static_cast(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:
us...@infra.apache.org




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

2021-01-07 Thread GitBox


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 
+
+#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 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:
us...@infra.apache.org




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

2021-01-07 Thread GitBox


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 
+
+#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 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 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:
us...@infra.apache.org




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

2021-01-07 Thread GitBox


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




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

2021-01-07 Thread GitBox


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 
+#include 
+#include 
+#include 
+
+#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{ 1 };
+  static constexpr char const* DEFAULT_MAX_POLL_TIME = "4 seconds";
+  static constexpr const std::size_t METADATA_COMMUNICATIONS_TIMEOUT_MS{ 6 
};
+
+  explicit ConsumeKafka(std::string name, utils::Identifier uuid = 
utils::Identifier()) :
+  Processor(name, uuid),
+  logger_(logging::LoggerFactory::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
+   * @

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

2021-01-07 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::PropertyBuilder::createProperty("Group ID")
+  ->wi

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

2021-01-07 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::PropertyBuilder::createProperty("Group ID")
+  ->wi

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

2021-01-07 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::PropertyBuilder::createProperty("Group ID")
+  ->wi

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

2021-01-07 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::PropertyBuilder::createProperty("Group ID")
+  ->wi

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

2021-01-07 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::PropertyBuilder::createProperty("Group ID")
+  ->wi

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

2021-01-07 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::PropertyBuilder::createProperty("Group ID")
+  ->wi

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

2021-01-07 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::PropertyBuilder::createProperty("Group ID")
+  ->wi

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

2021-01-07 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::PropertyBuilder::createProperty("Group ID")
+  ->wi

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

2021-01-07 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::PropertyBuilder::createProperty("Group ID")
+  ->wi

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

2021-01-07 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::PropertyBuilder::createProperty("Group ID")
+  ->wi

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

2021-01-07 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::PropertyBuilder::createProperty("Group ID")
+  ->wi

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

2021-01-07 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::PropertyBuilder::createProperty("Group ID")
+  ->wi

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

2021-01-07 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::PropertyBuilder::createProperty("Group ID")
+  ->wi

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

2021-01-07 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::PropertyBuilder::createProperty("Group ID")
+  ->wi

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

2021-01-07 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::PropertyBuilder::createProperty("Group ID")
+  ->wi

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

2021-01-07 Thread GitBox


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 
+#include 
+
+#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 :.")
+  ->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({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({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(true)
+  ->isRequired(true)
+  ->build());
+
+core::Property 
ConsumeKafka::GroupID(core::PropertyBuilder::createProperty("Group ID")
+  ->wi

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

2021-01-05 Thread GitBox


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(), 
"INFO");
-my_properties_->set("logger." + 
core::getClassName(), "DEBUG");
+my_properties_->set("logger." + 
core::getClassName(), "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:
us...@infra.apache.org




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

2021-01-05 Thread GitBox


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 
+#include 
+
+#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 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 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:
us...@infra.apache.org




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

2021-01-05 Thread GitBox


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 
+#include 
+#include 
+#include 
+
+#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:
us...@infra.apache.org




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

2020-12-10 Thread GitBox


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
+constexpr auto operator()(U&& v) const noexcept -> 
decltype(std::forward(v)) {
+return std::forward(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:
us...@infra.apache.org




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

2020-12-10 Thread GitBox


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




  1   2   >