You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by "lordgamez (via GitHub)" <gi...@apache.org> on 2023/06/13 08:57:22 UTC

[GitHub] [nifi-minifi-cpp] lordgamez opened a new pull request, #1587: MINIFICPP-2135 Add SSL support for Prometheus reporter

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

   https://issues.apache.org/jira/browse/MINIFICPP-2135
   
   --------------------------
   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 diff in pull request #1587: MINIFICPP-2135 Add SSL support for Prometheus reporter

Posted by "szaszm (via GitHub)" <gi...@apache.org>.
szaszm commented on code in PR #1587:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1587#discussion_r1228023558


##########
docker/test/integration/cluster/checkers/PrometheusChecker.py:
##########
@@ -18,7 +18,16 @@
 
 class PrometheusChecker:
     def __init__(self):
-        self.prometheus_client = PrometheusConnect(url="http://localhost:9090", disable_ssl=True)
+        self.use_ssl = False
+
+    def enable_ssl(self):
+        self.use_ssl = True
+
+    def _getClient(self):
+        if self.use_ssl:
+            return PrometheusConnect(url="https://localhost:9090", disable_ssl=True)

Review Comment:
   `disable_ssl=True`?



##########
docker/test/integration/cluster/containers/MinifiContainer.py:
##########
@@ -109,12 +110,16 @@ def _create_properties(self):
             if not self.options.enable_provenance:
                 f.write("nifi.provenance.repository.class.name=NoOpRepository\n")
 
-            if self.options.enable_prometheus:
+            if self.options.enable_prometheus or self.options.enable_prometheus_with_ssl:
                 f.write("nifi.metrics.publisher.agent.identifier=Agent1\n")
                 f.write("nifi.metrics.publisher.class=PrometheusMetricsPublisher\n")
                 f.write("nifi.metrics.publisher.PrometheusMetricsPublisher.port=9936\n")
                 f.write("nifi.metrics.publisher.metrics=RepositoryMetrics,QueueMetrics,PutFileMetrics,processorMetrics/Get.*,FlowInformation,DeviceInfoNode,AgentStatus\n")
 
+            if self.options.enable_prometheus_with_ssl:
+                f.write("nifi.metrics.publisher.PrometheusMetricsPublisher.certificate=/tmp/resources/prometheus-ssl/minifi-cpp-flow.crt\n")
+                f.write("nifi.metrics.publisher.PrometheusMetricsPublisher.ca.certificate=/tmp/resources/prometheus-ssl/root-ca.pem\n")

Review Comment:
   Why do we have both `minifi-cpp-flow.crt` and `prometheus.crt`? One TLS-enabled web server needs 1 key+cert pair.



##########
extensions/prometheus/PrometheusMetricsPublisher.cpp:
##########
@@ -33,18 +33,28 @@ PrometheusMetricsPublisher::PrometheusMetricsPublisher(const std::string &name,
 void PrometheusMetricsPublisher::initialize(const std::shared_ptr<Configure>& configuration, const std::shared_ptr<state::response::ResponseNodeLoader>& response_node_loader) {
   state::MetricsPublisher::initialize(configuration, response_node_loader);
   if (!exposer_) {
-    exposer_ = std::make_unique<PrometheusExposerWrapper>(readPort());
+    exposer_ = std::make_unique<PrometheusExposerWrapper>(readExposerConfig());
   }
   loadAgentIdentifier();
 }
 
-uint32_t PrometheusMetricsPublisher::readPort() {
+PrometheusExposerConfig PrometheusMetricsPublisher::readExposerConfig() const {
   gsl_Expects(configuration_);
+  PrometheusExposerConfig config;
   if (auto port = configuration_->get(Configuration::nifi_metrics_publisher_prometheus_metrics_publisher_port)) {
-    return std::stoul(*port);
+    config.port = std::stoul(*port);

Review Comment:
   `stoul` may throw, we should handle that 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.

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 diff in pull request #1587: MINIFICPP-2135 Add SSL support for Prometheus reporter

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on code in PR #1587:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1587#discussion_r1247539563


##########
docker/test/integration/features/MiNiFi_integration_test_driver.py:
##########
@@ -53,9 +53,9 @@ def __init__(self, context, feature_id: str):
         self.cluster.set_directory_bindings(self.docker_directory_bindings.get_directory_bindings(self.feature_id), self.docker_directory_bindings.get_data_directories(self.feature_id))
         self.root_ca_cert, self.root_ca_key = make_ca("root CA")
 
-        minifi_client_cert, minifi_client_key = make_client_cert(common_name=f"minifi-cpp-flow-{self.feature_id}",
-                                                                 ca_cert=self.root_ca_cert,
-                                                                 ca_key=self.root_ca_key)
+        minifi_client_cert, minifi_client_key = make_cert_without_extended_usage(common_name=f"minifi-cpp-flow-{self.feature_id}",

Review Comment:
   We already have a `make_server_cert` that has a `serverAuth` extended usage, so I would not mix it up with this version where we have no extended usage set.



-- 
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 diff in pull request #1587: MINIFICPP-2135 Add SSL support for Prometheus reporter

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on code in PR #1587:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1587#discussion_r1263819486


##########
METRICS.md:
##########
@@ -108,6 +108,15 @@ An agent identifier should also be defined to identify which agent the metric is
 
     nifi.metrics.publisher.agent.identifier=Agent1
 
+### Configure Prometheus metrics publisher with SSL
+
+The communication between MiNiFi and the Prometheus server can be encrypted using SSL. This can be achieved by adding the SSL certificate path (a single file containing both the certificate and the SSL key) and optionally adding the root CA path when using a self signed certificate to the minifi.properties file. Here is an example with the SSL properties:

Review Comment:
   In this scenario the MiNiFi publishes the metrics and acts as a server (using a CivetWeb server in the implementation) that can be scraped for the metrics by the Prometheus server. So in this case the MiNiFi needs a self signed certificate, not the Prometheus server.



-- 
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 diff in pull request #1587: MINIFICPP-2135 Add SSL support for Prometheus reporter

Posted by "szaszm (via GitHub)" <gi...@apache.org>.
szaszm commented on code in PR #1587:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1587#discussion_r1246666838


##########
docker/test/integration/cluster/checkers/PrometheusChecker.py:
##########
@@ -18,7 +18,16 @@
 
 class PrometheusChecker:
     def __init__(self):
-        self.prometheus_client = PrometheusConnect(url="http://localhost:9090", disable_ssl=True)
+        self.use_ssl = False
+
+    def enable_ssl(self):
+        self.use_ssl = True
+
+    def _getClient(self):
+        if self.use_ssl:
+            return PrometheusConnect(url="https://localhost:9090", disable_ssl=True)

Review Comment:
   Do we even need the option of using SSL on the prometheus listener? It doesn't interact with minifi, since prometheus is the client in the metrics collection, and it doesn't need client certificates either. Why don't we just throw out the ssl option in PrometheusChecker altogether?



-- 
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 diff in pull request #1587: MINIFICPP-2135 Add SSL support for Prometheus reporter

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on code in PR #1587:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1587#discussion_r1263847178


##########
METRICS.md:
##########
@@ -108,6 +108,15 @@ An agent identifier should also be defined to identify which agent the metric is
 
     nifi.metrics.publisher.agent.identifier=Agent1
 
+### Configure Prometheus metrics publisher with SSL
+
+The communication between MiNiFi and the Prometheus server can be encrypted using SSL. This can be achieved by adding the SSL certificate path (a single file containing both the certificate and the SSL key) and optionally adding the root CA path when using a self signed certificate to the minifi.properties file. Here is an example with the SSL properties:

Review Comment:
   Updated according our discussion in 45f72a651e07636f89cb177f8536a78cf3fe67d3



##########
METRICS.md:
##########
@@ -108,6 +108,15 @@ An agent identifier should also be defined to identify which agent the metric is
 
     nifi.metrics.publisher.agent.identifier=Agent1
 
+### Configure Prometheus metrics publisher with SSL
+
+The communication between MiNiFi and the Prometheus server can be encrypted using SSL. This can be achieved by adding the SSL certificate path (a single file containing both the certificate and the SSL key) and optionally adding the root CA path when using a self signed certificate to the minifi.properties file. Here is an example with the SSL properties:

Review Comment:
   Updated according to our discussion in 45f72a651e07636f89cb177f8536a78cf3fe67d3



-- 
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 diff in pull request #1587: MINIFICPP-2135 Add SSL support for Prometheus reporter

Posted by "szaszm (via GitHub)" <gi...@apache.org>.
szaszm commented on code in PR #1587:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1587#discussion_r1246671763


##########
docker/test/integration/features/MiNiFi_integration_test_driver.py:
##########
@@ -53,9 +53,9 @@ def __init__(self, context, feature_id: str):
         self.cluster.set_directory_bindings(self.docker_directory_bindings.get_directory_bindings(self.feature_id), self.docker_directory_bindings.get_data_directories(self.feature_id))
         self.root_ca_cert, self.root_ca_key = make_ca("root CA")
 
-        minifi_client_cert, minifi_client_key = make_client_cert(common_name=f"minifi-cpp-flow-{self.feature_id}",
-                                                                 ca_cert=self.root_ca_cert,
-                                                                 ca_key=self.root_ca_key)
+        minifi_client_cert, minifi_client_key = make_cert_without_extended_usage(common_name=f"minifi-cpp-flow-{self.feature_id}",

Review Comment:
   I think it would be more intuitive to refer to this as making server cert/key pair, and the `make_client_cert` could remain for client certificate generation.



-- 
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 diff in pull request #1587: MINIFICPP-2135 Add SSL support for Prometheus reporter

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on code in PR #1587:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1587#discussion_r1263846980


##########
extensions/prometheus/PrometheusExposerWrapper.cpp:
##########
@@ -20,9 +20,27 @@
 
 namespace org::apache::nifi::minifi::extensions::prometheus {
 
-PrometheusExposerWrapper::PrometheusExposerWrapper(uint32_t port)
-    : exposer_(std::to_string(port)) {
-  logger_->log_info("Started Prometheus metrics publisher on port %" PRIu32, port);
+PrometheusExposerWrapper::PrometheusExposerWrapper(const PrometheusExposerConfig& config)
+    : exposer_(parseExposerConfig(config)) {
+  logger_->log_info("Started Prometheus metrics publisher on port %" PRIu32, config.port);

Review Comment:
   Updated in 45f72a651e07636f89cb177f8536a78cf3fe67d3



##########
docker/test/integration/cluster/containers/PrometheusContainer.py:
##########
@@ -16,13 +16,53 @@
 import os
 import tempfile
 import docker.types
+
 from .Container import Container
+from OpenSSL import crypto
+from ssl_utils.SSL_cert_utils import make_cert_without_extended_usage
 
 
 class PrometheusContainer(Container):
-    def __init__(self, feature_context, name, vols, network, image_store, command=None):
-        super().__init__(feature_context, name, 'prometheus', vols, network, image_store, command)
-        prometheus_yml_content = """
+    def __init__(self, feature_context, name, vols, network, image_store, command=None, ssl=False):
+        engine = "prometheus-ssl" if ssl else "prometheus"
+        super().__init__(feature_context, name, engine, vols, network, image_store, command)
+        self.ssl = ssl
+        if ssl:
+            prometheus_cert, prometheus_key = make_cert_without_extended_usage(f"prometheus-{feature_context.id}", feature_context.root_ca_cert, feature_context.root_ca_key)
+
+            self.root_ca_file = tempfile.NamedTemporaryFile(delete=False)
+            self.root_ca_file.write(crypto.dump_certificate(type=crypto.FILETYPE_PEM, cert=feature_context.root_ca_cert))
+            self.root_ca_file.close()
+            os.chmod(self.root_ca_file.name, 0o644)
+
+            self.prometheus_cert_file = tempfile.NamedTemporaryFile(delete=False)
+            self.prometheus_cert_file.write(crypto.dump_certificate(type=crypto.FILETYPE_PEM, cert=prometheus_cert))
+            self.prometheus_cert_file.close()
+            os.chmod(self.prometheus_cert_file.name, 0o644)
+
+            self.prometheus_key_file = tempfile.NamedTemporaryFile(delete=False)
+            self.prometheus_key_file.write(crypto.dump_privatekey(type=crypto.FILETYPE_PEM, pkey=prometheus_key))
+            self.prometheus_key_file.close()
+            os.chmod(self.prometheus_key_file.name, 0o644)
+
+            prometheus_yml_content = """
+global:
+  scrape_interval: 2s
+  evaluation_interval: 15s
+scrape_configs:
+  - job_name: "minifi"
+    static_configs:
+      - targets: ["minifi-cpp-flow-{feature_id}:9936"]
+    scheme: https
+    tls_config:
+      ca_file: /etc/prometheus/certs/root-ca.pem
+""".format(feature_id=self.feature_context.id)
+            self.yaml_file = tempfile.NamedTemporaryFile(delete=False)
+            self.yaml_file.write(prometheus_yml_content.encode())
+            self.yaml_file.close()
+            os.chmod(self.yaml_file.name, 0o644)
+        else:
+            prometheus_yml_content = """

Review Comment:
   Updated in 45f72a651e07636f89cb177f8536a78cf3fe67d3



-- 
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 #1587: MINIFICPP-2135 Add SSL support for Prometheus reporter

Posted by "szaszm (via GitHub)" <gi...@apache.org>.
szaszm closed pull request #1587: MINIFICPP-2135 Add SSL support for Prometheus reporter
URL: https://github.com/apache/nifi-minifi-cpp/pull/1587


-- 
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 diff in pull request #1587: MINIFICPP-2135 Add SSL support for Prometheus reporter

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on code in PR #1587:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1587#discussion_r1229154144


##########
docker/test/integration/features/MiNiFi_integration_test_driver.py:
##########
@@ -53,9 +53,9 @@ def __init__(self, context, feature_id: str):
         self.cluster.set_directory_bindings(self.docker_directory_bindings.get_directory_bindings(self.feature_id), self.docker_directory_bindings.get_data_directories(self.feature_id))
         self.root_ca_cert, self.root_ca_key = make_ca("root CA")
 
-        minifi_client_cert, minifi_client_key = make_client_cert(common_name=f"minifi-cpp-flow-{self.feature_id}",
-                                                                 ca_cert=self.root_ca_cert,
-                                                                 ca_key=self.root_ca_key)
+        minifi_client_cert, minifi_client_key = make_cert_without_extended_usage(common_name=f"minifi-cpp-flow-{self.feature_id}",

Review Comment:
   In Prometheus tests MiNiFi does not work as a client, and setting its SSL certificate with `clientAuth` extended usage fails with the following error: `x509: certificate specifies an incompatible key usage`
   Because of the `clientAuth` is removed 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.

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 diff in pull request #1587: MINIFICPP-2135 Add SSL support for Prometheus reporter

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on code in PR #1587:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1587#discussion_r1235332846


##########
docker/test/integration/cluster/checkers/PrometheusChecker.py:
##########
@@ -18,7 +18,16 @@
 
 class PrometheusChecker:
     def __init__(self):
-        self.prometheus_client = PrometheusConnect(url="http://localhost:9090", disable_ssl=True)
+        self.use_ssl = False
+
+    def enable_ssl(self):
+        self.use_ssl = True
+
+    def _getClient(self):
+        if self.use_ssl:
+            return PrometheusConnect(url="https://localhost:9090", disable_ssl=True)

Review Comment:
   We don't want to use SSL verification for the python client when connecting from this checker. We only need SSL verification between the MiNiFi and the Prometheus server.



##########
docker/test/integration/cluster/containers/MinifiContainer.py:
##########
@@ -109,12 +110,16 @@ def _create_properties(self):
             if not self.options.enable_provenance:
                 f.write("nifi.provenance.repository.class.name=NoOpRepository\n")
 
-            if self.options.enable_prometheus:
+            if self.options.enable_prometheus or self.options.enable_prometheus_with_ssl:
                 f.write("nifi.metrics.publisher.agent.identifier=Agent1\n")
                 f.write("nifi.metrics.publisher.class=PrometheusMetricsPublisher\n")
                 f.write("nifi.metrics.publisher.PrometheusMetricsPublisher.port=9936\n")
                 f.write("nifi.metrics.publisher.metrics=RepositoryMetrics,QueueMetrics,PutFileMetrics,processorMetrics/Get.*,FlowInformation,DeviceInfoNode,AgentStatus\n")
 
+            if self.options.enable_prometheus_with_ssl:
+                f.write("nifi.metrics.publisher.PrometheusMetricsPublisher.certificate=/tmp/resources/prometheus-ssl/minifi-cpp-flow.crt\n")
+                f.write("nifi.metrics.publisher.PrometheusMetricsPublisher.ca.certificate=/tmp/resources/prometheus-ssl/root-ca.pem\n")

Review Comment:
   In this case MiNiFi works as a web server as it exposes metrics through a CivetWeb server implemented by the prometheus cpp library. Civetweb requires a merged crt+key file. For communication over SSL the Prometheus server also needs to be configured with a separate key and a certificate pair.



##########
extensions/prometheus/PrometheusMetricsPublisher.cpp:
##########
@@ -33,18 +33,28 @@ PrometheusMetricsPublisher::PrometheusMetricsPublisher(const std::string &name,
 void PrometheusMetricsPublisher::initialize(const std::shared_ptr<Configure>& configuration, const std::shared_ptr<state::response::ResponseNodeLoader>& response_node_loader) {
   state::MetricsPublisher::initialize(configuration, response_node_loader);
   if (!exposer_) {
-    exposer_ = std::make_unique<PrometheusExposerWrapper>(readPort());
+    exposer_ = std::make_unique<PrometheusExposerWrapper>(readExposerConfig());
   }
   loadAgentIdentifier();
 }
 
-uint32_t PrometheusMetricsPublisher::readPort() {
+PrometheusExposerConfig PrometheusMetricsPublisher::readExposerConfig() const {
   gsl_Expects(configuration_);
+  PrometheusExposerConfig config;
   if (auto port = configuration_->get(Configuration::nifi_metrics_publisher_prometheus_metrics_publisher_port)) {
-    return std::stoul(*port);
+    config.port = std::stoul(*port);

Review Comment:
   Updated in 7a0581681d7938a60feb31944baa96415abb4d64



-- 
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 diff in pull request #1587: MINIFICPP-2135 Add SSL support for Prometheus reporter

Posted by "szaszm (via GitHub)" <gi...@apache.org>.
szaszm commented on code in PR #1587:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1587#discussion_r1246669182


##########
docker/test/integration/cluster/containers/MinifiContainer.py:
##########
@@ -109,12 +110,16 @@ def _create_properties(self):
             if not self.options.enable_provenance:
                 f.write("nifi.provenance.repository.class.name=NoOpRepository\n")
 
-            if self.options.enable_prometheus:
+            if self.options.enable_prometheus or self.options.enable_prometheus_with_ssl:
                 f.write("nifi.metrics.publisher.agent.identifier=Agent1\n")
                 f.write("nifi.metrics.publisher.class=PrometheusMetricsPublisher\n")
                 f.write("nifi.metrics.publisher.PrometheusMetricsPublisher.port=9936\n")
                 f.write("nifi.metrics.publisher.metrics=RepositoryMetrics,QueueMetrics,PutFileMetrics,processorMetrics/Get.*,FlowInformation,DeviceInfoNode,AgentStatus\n")
 
+            if self.options.enable_prometheus_with_ssl:
+                f.write("nifi.metrics.publisher.PrometheusMetricsPublisher.certificate=/tmp/resources/prometheus-ssl/minifi-cpp-flow.crt\n")
+                f.write("nifi.metrics.publisher.PrometheusMetricsPublisher.ca.certificate=/tmp/resources/prometheus-ssl/root-ca.pem\n")

Review Comment:
   I suggested throwing out prometheus server side ssl in my other reply, just noting on this thread as well. If we do that, extra certs are no longer needed.



-- 
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 diff in pull request #1587: MINIFICPP-2135 Add SSL support for Prometheus reporter

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on code in PR #1587:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1587#discussion_r1247640156


##########
docker/test/integration/cluster/checkers/PrometheusChecker.py:
##########
@@ -18,7 +18,16 @@
 
 class PrometheusChecker:
     def __init__(self):
-        self.prometheus_client = PrometheusConnect(url="http://localhost:9090", disable_ssl=True)
+        self.use_ssl = False
+
+    def enable_ssl(self):
+        self.use_ssl = True
+
+    def _getClient(self):
+        if self.use_ssl:
+            return PrometheusConnect(url="https://localhost:9090", disable_ssl=True)

Review Comment:
   You are right, we don't need it in the Prometheus server, I removed the SSL config from the server in f360282ef71dcf9645d90dba3f60a5140f8773aa



##########
docker/test/integration/cluster/containers/MinifiContainer.py:
##########
@@ -109,12 +110,16 @@ def _create_properties(self):
             if not self.options.enable_provenance:
                 f.write("nifi.provenance.repository.class.name=NoOpRepository\n")
 
-            if self.options.enable_prometheus:
+            if self.options.enable_prometheus or self.options.enable_prometheus_with_ssl:
                 f.write("nifi.metrics.publisher.agent.identifier=Agent1\n")
                 f.write("nifi.metrics.publisher.class=PrometheusMetricsPublisher\n")
                 f.write("nifi.metrics.publisher.PrometheusMetricsPublisher.port=9936\n")
                 f.write("nifi.metrics.publisher.metrics=RepositoryMetrics,QueueMetrics,PutFileMetrics,processorMetrics/Get.*,FlowInformation,DeviceInfoNode,AgentStatus\n")
 
+            if self.options.enable_prometheus_with_ssl:
+                f.write("nifi.metrics.publisher.PrometheusMetricsPublisher.certificate=/tmp/resources/prometheus-ssl/minifi-cpp-flow.crt\n")
+                f.write("nifi.metrics.publisher.PrometheusMetricsPublisher.ca.certificate=/tmp/resources/prometheus-ssl/root-ca.pem\n")

Review Comment:
   Updated in f360282ef71dcf9645d90dba3f60a5140f8773aa



-- 
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] fgerlits commented on a diff in pull request #1587: MINIFICPP-2135 Add SSL support for Prometheus reporter

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1587:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1587#discussion_r1263391342


##########
METRICS.md:
##########
@@ -108,6 +108,15 @@ An agent identifier should also be defined to identify which agent the metric is
 
     nifi.metrics.publisher.agent.identifier=Agent1
 
+### Configure Prometheus metrics publisher with SSL
+
+The communication between MiNiFi and the Prometheus server can be encrypted using SSL. This can be achieved by adding the SSL certificate path (a single file containing both the certificate and the SSL key) and optionally adding the root CA path when using a self signed certificate to the minifi.properties file. Here is an example with the SSL properties:

Review Comment:
   The root CA is for the server certificate, isn't it?  I would make that clearer:
   ```suggestion
   The communication between MiNiFi and the Prometheus server can be encrypted using SSL. This can be achieved by adding the SSL certificate path (a single file containing both the client certificate and the client SSL key) and optionally adding the root CA path if the Prometheus server uses a self-signed certificate, to the minifi.properties file. Here is an example with the SSL properties:
   ```



##########
extensions/prometheus/PrometheusExposerWrapper.cpp:
##########
@@ -20,9 +20,27 @@
 
 namespace org::apache::nifi::minifi::extensions::prometheus {
 
-PrometheusExposerWrapper::PrometheusExposerWrapper(uint32_t port)
-    : exposer_(std::to_string(port)) {
-  logger_->log_info("Started Prometheus metrics publisher on port %" PRIu32, port);
+PrometheusExposerWrapper::PrometheusExposerWrapper(const PrometheusExposerConfig& config)
+    : exposer_(parseExposerConfig(config)) {
+  logger_->log_info("Started Prometheus metrics publisher on port %" PRIu32, config.port);

Review Comment:
   It could be useful to add "with TLS enabled" to this log message if the config contains a certificate.



##########
docker/test/integration/cluster/containers/PrometheusContainer.py:
##########
@@ -16,13 +16,53 @@
 import os
 import tempfile
 import docker.types
+
 from .Container import Container
+from OpenSSL import crypto
+from ssl_utils.SSL_cert_utils import make_cert_without_extended_usage
 
 
 class PrometheusContainer(Container):
-    def __init__(self, feature_context, name, vols, network, image_store, command=None):
-        super().__init__(feature_context, name, 'prometheus', vols, network, image_store, command)
-        prometheus_yml_content = """
+    def __init__(self, feature_context, name, vols, network, image_store, command=None, ssl=False):
+        engine = "prometheus-ssl" if ssl else "prometheus"
+        super().__init__(feature_context, name, engine, vols, network, image_store, command)
+        self.ssl = ssl
+        if ssl:
+            prometheus_cert, prometheus_key = make_cert_without_extended_usage(f"prometheus-{feature_context.id}", feature_context.root_ca_cert, feature_context.root_ca_key)
+
+            self.root_ca_file = tempfile.NamedTemporaryFile(delete=False)
+            self.root_ca_file.write(crypto.dump_certificate(type=crypto.FILETYPE_PEM, cert=feature_context.root_ca_cert))
+            self.root_ca_file.close()
+            os.chmod(self.root_ca_file.name, 0o644)
+
+            self.prometheus_cert_file = tempfile.NamedTemporaryFile(delete=False)
+            self.prometheus_cert_file.write(crypto.dump_certificate(type=crypto.FILETYPE_PEM, cert=prometheus_cert))
+            self.prometheus_cert_file.close()
+            os.chmod(self.prometheus_cert_file.name, 0o644)
+
+            self.prometheus_key_file = tempfile.NamedTemporaryFile(delete=False)
+            self.prometheus_key_file.write(crypto.dump_privatekey(type=crypto.FILETYPE_PEM, pkey=prometheus_key))
+            self.prometheus_key_file.close()
+            os.chmod(self.prometheus_key_file.name, 0o644)
+
+            prometheus_yml_content = """
+global:
+  scrape_interval: 2s
+  evaluation_interval: 15s
+scrape_configs:
+  - job_name: "minifi"
+    static_configs:
+      - targets: ["minifi-cpp-flow-{feature_id}:9936"]
+    scheme: https
+    tls_config:
+      ca_file: /etc/prometheus/certs/root-ca.pem
+""".format(feature_id=self.feature_context.id)
+            self.yaml_file = tempfile.NamedTemporaryFile(delete=False)
+            self.yaml_file.write(prometheus_yml_content.encode())
+            self.yaml_file.close()
+            os.chmod(self.yaml_file.name, 0o644)
+        else:
+            prometheus_yml_content = """

Review Comment:
   This looks a bit confusing to me.  I think setting something like `extra_ssl_settings` to either something or nothing, and then having a single
   ```python
           prometheus_yml_content = """
   global:
     scrape_interval: 2s
     evaluation_interval: 15s
   scrape_configs:
     - job_name: "minifi"
       static_configs:
         - targets: ["minifi-cpp-flow-{feature_id}:9936"]
   {extra_ssl_settings}
   """.format(...)
   ```
   would be more readable.



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