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/09 16:17:49 UTC

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

lordgamez opened a new pull request #1212:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1212


   - Add new security options: SASL_PLAIN and SASL_SSL
   - Add SASL mechanism property with GSSAPI and PLAIN support
   - Add Username and Password properties for PLAIN mechanism
   - Extract common Kafka authentication code to add the same authentication options to ConsumeKafka processor as PublishKafka
   - Add Kafka broker configuration and tests for SASL/PLAIN and SASL/SSL with username and password authentication
   
   https://issues.apache.org/jira/browse/MINIFICPP-1677
   
   -----------------------------------------------------------------------------------------------------
   Thank you for submitting a contribution to Apache NiFi - MiNiFi C++.
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [ ] Is there a JIRA ticket associated with this PR? Is it referenced
        in the commit message?
   
   - [ ] Does your PR title start with MINIFICPP-XXXX where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [ ] Has your PR been rebased against the latest commit within the target branch (typically main)?
   
   - [ ] Is your initial contribution a single, squashed commit?
   
   ### For code changes:
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
   - [ ] If applicable, have you updated the LICENSE file?
   - [ ] If applicable, have you updated the NOTICE file?
   
   ### For documentation related changes:
   - [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
   
   ### Note:
   Please ensure that once the PR is submitted, you check GitHub Actions CI results for build issues and submit an update to your PR as soon as possible.
   


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



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

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



##########
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:
       Doesn't this prevent resetting a config value?

##########
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:
       Are 9095 and 29095 intentionally not exposed? Or am I missing something?

##########
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 don't see why `context` is now passed as a pointer. I'm assuming getProperty required non-const reference to ProcessContext, but it's still UB to call this with a null pointer. Can we make it into a reference parameter?

##########
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:
       Can we make these `constexpr` and have a simpler type, like `const char[]`, `const char* const` or `std::string_view`? I think it's better to avoid memory allocation during static initialization unless it's really needed.

##########
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:
       Here as well, `context` should be a reference. Maybe it could even be a reference to `const`?

##########
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:
       Can we make the `context` parameter a reference instead of a pointer? It's UB to call this with null, so we might as well reflect it in the signature itself.




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



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

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



##########
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:
       Fixed in 23ad6012bd8b7c4ea91a35182b239248dfa6f813




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



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

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



##########
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:
       Added `gsl::not_null` in a762c5c5c238fbd8cf383205cdf2d87932b3c741




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



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

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



##########
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:
       Nitpicking: consider making `config` a reference or `gsl::not_null<...>`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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



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

Posted by GitBox <gi...@apache.org>.
lordgamez commented on pull request #1212:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1212#issuecomment-971523030


   > `const` missing from here compared to the implementation.
   
   Fixed in 23ad6012bd8b7c4ea91a35182b239248dfa6f813


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



[GitHub] [nifi-minifi-cpp] szaszm closed pull request #1212: MINIFICPP-1677 Add SASL PLAIN mechanism support to Kafka processors

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


   


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



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

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



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

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



##########
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:
       `constexpr` implies implicit top-level `const`, so `constexpr const char* const` is the same as `constexpr const char*`. I prefer the last const to be removed, but this is just nitpicking.




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