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

[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1212: MINIFICPP-1677 Add SASL PLAIN mechanism support to Kafka processors

lordgamez commented on a change in pull request #1212:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1212#discussion_r751109546



##########
File path: docker/test/integration/minifi/core/KafkaBrokerContainer.py
##########
@@ -19,15 +19,15 @@ def deploy(self):
             detach=True,
             name='kafka-broker',
             network=self.network.name,
-            ports={'9092/tcp': 9092, '29092/tcp': 29092, '9093/tcp': 9093, '29093/tcp': 29093},
+            ports={'9092/tcp': 9092, '29092/tcp': 29092, '9093/tcp': 9093, '29093/tcp': 29093, '9094/tcp': 9094, '29094/tcp': 29094},

Review comment:
       It was missing, only it did not have any impact because we did not have any operations on that port from the host machine. I added it in 98ed8e2450436c2502a71f0bc99301fa4576db2c

##########
File path: extensions/librdkafka/rdkafka_utils.cpp
##########
@@ -29,6 +29,9 @@ namespace minifi {
 namespace utils {
 
 void setKafkaConfigurationField(rd_kafka_conf_t& configuration, const std::string& field_name, const std::string& value) {
+  if (value.empty()) {
+    return;
+  }

Review comment:
       Good point, I just wanted to prevent any unnecessary config settings, but it should not be a problem when an empty value is set, removed in 98ed8e2450436c2502a71f0bc99301fa4576db2c

##########
File path: extensions/librdkafka/PublishKafka.h
##########
@@ -89,8 +84,19 @@ class PublishKafka : public core::Processor {
   EXTENSIONAPI static const core::Relationship Failure;
   EXTENSIONAPI static const core::Relationship Success;
 
+  static const std::string COMPRESSION_CODEC_NONE;
+  static const std::string COMPRESSION_CODEC_GZIP;
+  static const std::string COMPRESSION_CODEC_SNAPPY;
+  static const std::string ROUND_ROBIN_PARTITIONING;
+  static const std::string RANDOM_PARTITIONING;
+  static const std::string USER_DEFINED_PARTITIONING;
+  static const std::string DELIVERY_REPLICATED;
+  static const std::string DELIVERY_ONE_NODE;
+  static const std::string DELIVERY_BEST_EFFORT;
+  static const std::string KAFKA_KEY_ATTRIBUTE;

Review comment:
       Changed in 98ed8e2450436c2502a71f0bc99301fa4576db2c

##########
File path: extensions/librdkafka/ConsumeKafka.h
##########
@@ -126,7 +124,7 @@ class ConsumeKafka : public core::Processor {
  private:
   void create_topic_partition_list();
   void extend_config_from_dynamic_properties(const core::ProcessContext& context);
-  void configure_new_connection(const core::ProcessContext& context);
+  void configure_new_connection(core::ProcessContext* context);

Review comment:
       I wanted to align it with the parameters of the processor config utils functions, but it does make more sense to have all of them as references instead of pointers. Updated in 98ed8e2450436c2502a71f0bc99301fa4576db2c

##########
File path: libminifi/include/utils/ProcessorConfigUtils.h
##########
@@ -30,7 +30,15 @@ namespace nifi {
 namespace minifi {
 namespace utils {
 
-std::string getRequiredPropertyOrThrow(const core::ProcessContext* context, const std::string& property_name);
+template<typename PropertyType = std::string>
+PropertyType getRequiredPropertyOrThrow(const core::ProcessContext* context, const std::string& property_name) {

Review comment:
       Changed in 98ed8e2450436c2502a71f0bc99301fa4576db2c

##########
File path: extensions/librdkafka/KafkaProcessorBase.cpp
##########
@@ -0,0 +1,135 @@
+/**
+ * @file KafkaProcessorBase.cpp
+ * KafkaProcessorBase class implementation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 "KafkaProcessorBase.h"
+
+#include "rdkafka_utils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org::apache::nifi::minifi::processors {
+
+const core::Property KafkaProcessorBase::SecurityProtocol(
+        core::PropertyBuilder::createProperty("Security Protocol")
+        ->withDescription("Protocol used to communicate with brokers. Corresponds to Kafka's 'security.protocol' property.")
+        ->withDefaultValue<std::string>(toString(SecurityProtocolOption::PLAINTEXT))
+        ->withAllowableValues<std::string>(SecurityProtocolOption::values())
+        ->isRequired(true)
+        ->build());
+const core::Property KafkaProcessorBase::SSLContextService(
+    core::PropertyBuilder::createProperty("SSL Context Service")
+        ->withDescription("SSL Context Service Name")
+        ->asType<minifi::controllers::SSLContextService>()
+        ->build());
+const core::Property KafkaProcessorBase::KerberosServiceName(
+    core::PropertyBuilder::createProperty("Kerberos Service Name")
+        ->withDescription("Kerberos Service Name")
+        ->build());
+const core::Property KafkaProcessorBase::KerberosPrincipal(
+    core::PropertyBuilder::createProperty("Kerberos Principal")
+        ->withDescription("Keberos Principal")
+        ->build());
+const core::Property KafkaProcessorBase::KerberosKeytabPath(
+    core::PropertyBuilder::createProperty("Kerberos Keytab Path")
+        ->withDescription("The path to the location on the local filesystem where the kerberos keytab is located. Read permission on the file is required.")
+        ->build());
+const core::Property KafkaProcessorBase::SASLMechanism(
+        core::PropertyBuilder::createProperty("SASL Mechanism")
+        ->withDescription("The SASL mechanism to use for authentication. Corresponds to Kafka's 'sasl.mechanism' property.")
+        ->withDefaultValue<std::string>(toString(SASLMechanismOption::GSSAPI))
+        ->withAllowableValues<std::string>(SASLMechanismOption::values())
+        ->isRequired(true)
+        ->build());
+const core::Property KafkaProcessorBase::Username(
+    core::PropertyBuilder::createProperty("Username")
+        ->withDescription("The username when the SASL Mechanism is sasl_plaintext")
+        ->build());
+const core::Property KafkaProcessorBase::Password(
+    core::PropertyBuilder::createProperty("Password")
+        ->withDescription("The password for the given username when the SASL Mechanism is sasl_plaintext")
+        ->build());
+
+std::optional<utils::SSL_data> KafkaProcessorBase::getSslData(core::ProcessContext* context) const {
+  std::string ssl_service_name;
+  if (context->getProperty(SSLContextService.getName(), ssl_service_name) && !ssl_service_name.empty()) {
+    std::shared_ptr<core::controller::ControllerService> service = context->getControllerService(ssl_service_name);
+    if (service) {
+      auto ssl_service = std::static_pointer_cast<minifi::controllers::SSLContextService>(service);
+      utils::SSL_data ssl_data;
+      ssl_data.ca_loc = ssl_service->getCACertificate();
+      ssl_data.cert_loc = ssl_service->getCertificateFile();
+      ssl_data.key_loc = ssl_service->getPrivateKeyFile();
+      ssl_data.key_pw = ssl_service->getPassphrase();
+      return ssl_data;
+    } else {
+      logger_->log_warn("SSL Context Service property is set to '%s', but the controller service could not be found.", ssl_service_name);
+      return std::nullopt;
+    }
+  } else if (security_protocol_ == SecurityProtocolOption::SSL || security_protocol_ == SecurityProtocolOption::SASL_SSL) {
+    logger_->log_warn("Security protocol is set to %s, but no valid SSL Context Service property is set.", security_protocol_.toString());
+  }
+
+  return std::nullopt;
+}
+
+void KafkaProcessorBase::setKafkaAuthenticationParameters(core::ProcessContext* context, rd_kafka_conf_t* config) {

Review comment:
       Updated in 98ed8e2450436c2502a71f0bc99301fa4576db2c. Unfortunately it cannot be const because `getControllerService` function is non-const. 




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org