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 2022/05/26 21:09:08 UTC

[GitHub] [nifi-minifi-cpp] lordgamez opened a new pull request, #1340: MINIFICPP-1829 Export metrics for use with Prometheus

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

   This feature enables the agent to expose metrics through a Prometheus metrics publisher to be scraped when configured
   
   - Metric loading is extracted from C2 client to be reused
   - C2 metrics are tested and fixed where needed
   - C2 update configuration is tested to verify metric reloading
   - Metrics are extended with calculateMetrics method for Prometheus or any future monitoring tool to be published with new Prometheus extension
   - prometheus-cpp client is added for exposing metrics to Prometheus
   - minifi.properties is extended to specify metrics publisher and published metric nodes
   
   https://issues.apache.org/jira/browse/MINIFICPP-1829
   
   ----------------------------------------------------------
   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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r901852584


##########
libminifi/include/properties/Configuration.h:
##########
@@ -156,12 +156,17 @@ class Configuration : public Properties {
 
   static constexpr const char *nifi_asset_directory = "nifi.asset.directory";
 
+  // Metrics publisher options
+  static constexpr const char *nifi_metrics_publisher_class = "nifi.metrics.publisher.class";
+  static constexpr const char *nifi_metrics_publisher_prometheus_metrics_publisher_port = "nifi.metrics.publisher.PrometheusMetricsPublisher.port";

Review Comment:
   Any way to move this config property definition to the prometheus extension?



##########
METRICS.md:
##########
@@ -0,0 +1,155 @@
+<!--
+  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.
+-->
+
+# Apache NiFi - MiNiFi - C++ Metrics Readme.
+
+
+This readme defines the metrics published by Apache NiFi. All options defined are located in minifi.properties.
+
+## Table of Contents
+
+- [Description](#description)
+- [Configuration](#configuration)
+- [Metrics](#metrics)
+
+## Description
+
+Apache NiFi MiNiFi C++ can communicate metrics about the agent's status, that can be a system level or component level metric.
+These metrics are exposed through the agent implemented metric publishers that can be configured in the minifi.properties.
+Aside from the publisher exposed metrics, metrics are also sent through C2 protocol of which there is more information in the
+[C2 documentation](C2.md#metrics).
+
+## Configuration
+
+To configure the a metrics publisher first we have to set which publisher class should be used:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.class=PrometheusMetricsPublisher
+
+Currently PrometheusMetricsPublisher is the only available publisher in MiNiFi C++ which publishes metrics to a Prometheus server.
+To use the publisher a port should also be configured where the metrics will be available to be scraped through:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.PrometheusMetricsPublisher.port=9936
+
+The last option defines which metric classes should be exposed through the metrics publisher in configured with a comma separated value:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.metrics=QueueMetrics,RepositoryMetrics,GetFileMetrics,DeviceInfoNode,FlowInformation
+
+## Metrics
+
+The following section defines the currently available metrics to be published by the MiNiFi C++ agent.
+
+NOTE: In Prometheus all metrics are extended with a `minifi_` prefix to mark the domain of the metric. For example the `connection_name` metric is published as `minifi_connection_name` in Prometheus.
+
+### QueueMetrics
+
+QueueMetrics is a system level metric that reports queue metrics for every connection in the flow.
+
+| Metric name          | Labels                                         | Description                                |
+|----------------------|------------------------------------------------|--------------------------------------------|
+| queue_data_size      | metric_class, connection_uuid, connection_name | Max queue size to apply back pressure      |
+| queue_data_size_max  | metric_class, connection_uuid, connection_name | Max queue data size to apply back pressure |
+| queue_size           | metric_class, connection_uuid, connection_name | Current queue size                         |
+| queue_size_max       | metric_class, connection_uuid, connection_name | Current queue data size                    |

Review Comment:
   I've found this guide to metric naming: https://prometheus.io/docs/practices/naming/
   
   By the way I think the descriptions are not in the correct order here.



##########
libminifi/include/core/state/nodes/RepositoryMetrics.h:
##########
@@ -90,15 +87,18 @@ class RepositoryMetrics : public ResponseNode {
     return serialized;
   }
 
+  std::vector<PublishedMetric> calculateMetrics() override {
+    std::vector<PublishedMetric> metrics;
+    for (const auto& [_, repo] : repositories_) {
+      metrics.push_back({"is_running", (repo->isRunning() ? 1.0 : 0.0), {{"metric_class", getName()}, {"repository_name", repo->getName()}}});
+      metrics.push_back({"is_full", (repo->isFull() ? 1.0 : 0.0), {{"metric_class", getName()}, {"repository_name", repo->getName()}}});
+      metrics.push_back({"repository_size", static_cast<double>(repo->getRepoSize()), {{"metric_class", getName()}, {"repository_name", repo->getName()}}});
+    }
+    return metrics;
+  }

Review Comment:
   I feel like the logic here and in `serialize` would be better, if they were unified for all metric nodes, with a common mapping step for prometheus-style transformation, that could happen in prometheus extension code.



-- 
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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r899862647


##########
METRICS.md:
##########
@@ -0,0 +1,153 @@
+<!--
+  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.
+-->
+
+# Apache NiFi - MiNiFi - C++ Metrics Readme.
+
+
+This readme defines the metrics published by Apache NiFi. All options defined are located in minifi.properties.
+
+## Table of Contents
+
+- [Description](#description)
+- [Configuration](#configuration)
+- [Metrics](#metrics)
+
+## Description
+
+Apache NiFi MiNiFi C++ can communicate metrics about the agent's status, that can be a system level or component level metric.
+These metrics are exposed through the agent implemented metric publishers that can be configured in the minifi.properties.
+Aside from the publisher exposed metrics, metrics are also sent through C2 protocol of which there is more information in the
+[C2 documentation](C2.md#metrics).
+
+## Configuration
+
+To configure the a metrics publisher first we have to set which publisher class should be used:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.class=PrometheusMetricsPublisher
+
+Currently PrometheusMetricsPublisher is the only available publisher in MiNiFi C++ which publishes metrics to a Prometheus server.
+To use the publisher a port should also be configured where the metrics will be available to be scraped through:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.port=9936

Review Comment:
   Updated in 2f43e4e6818e72cc2e6cd5a0e22d4f96962db9eb



-- 
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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r905075513


##########
libminifi/src/c2/C2Client.cpp:
##########
@@ -206,35 +130,13 @@ void C2Client::loadC2ResponseConfiguration(const std::string &prefix) {
       }
       std::shared_ptr<state::response::ResponseNode> new_node = std::make_shared<state::response::ObjectNode>(name);
       if (configuration_->get(classOption, class_definitions)) {
-        std::vector<std::string> classes = utils::StringUtils::split(class_definitions, ",");
-        for (const std::string& clazz : classes) {
-          // instantiate the object
-          std::shared_ptr<core::CoreComponent> ptr = core::ClassLoader::getDefaultClassLoader().instantiate(clazz, clazz);
-          if (nullptr == ptr) {
-            const bool found_metric = [&] {
-              std::lock_guard<std::mutex> guard{metrics_mutex_};
-              auto metric = component_metrics_.find(clazz);
-              if (metric != component_metrics_.end()) {
-                ptr = metric->second;
-                return true;
-              }
-              return false;
-            }();
-            if (!found_metric) {
-              logger_->log_error("No metric defined for %s", clazz);
-              continue;
-            }
-          }
-          auto node = std::dynamic_pointer_cast<state::response::ResponseNode>(ptr);
-          std::static_pointer_cast<state::response::ObjectNode>(new_node)->add_node(node);
-        }
-
+        loadNodeClasses(class_definitions, new_node);
       } else {
         std::string optionName = option + "." + name;
-        auto node = loadC2ResponseConfiguration(optionName, new_node);
+        loadC2ResponseConfiguration(optionName, new_node);
       }
 
-      std::lock_guard<std::mutex> guard{metrics_mutex_};
+      // We don't need to lock here we do it in the initializeResponseNodes

Review Comment:
   Consider adding a comma. I'm not good a grammar, but I think it helps readability by separating the parts of the sentence.
   ```suggestion
         // We don't need to lock here, we do it in the initializeResponseNodes
   ```



##########
libminifi/include/core/state/ConnectionStore.h:
##########
@@ -0,0 +1,60 @@
+/**
+ *
+ * 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 <map>
+#include <string>
+#include <vector>
+
+#include "Connection.h"
+#include "utils/gsl.h"
+
+namespace org::apache::nifi::minifi::state {
+
+class ConnectionStore {
+ public:
+  void updateConnection(minifi::Connection* connection) {
+    if (nullptr != connection) {
+      connections_[connection->getUUIDStr()] = connection;
+    }
+  }
+
+  std::vector<PublishedMetric> calculateConnectionMetrics(const std::string& metric_class) {
+    std::vector<PublishedMetric> metrics;
+
+    for (const auto& [_, connection] : connections_) {
+      metrics.push_back({"queue_data_size", static_cast<double>(connection->getQueueDataSize()),
+        {{"connection_uuid", connection->getUUIDStr()}, {"connection_name", connection->getName()}, {"metric_class", metric_class}}});
+      metrics.push_back({"queue_data_size_max", static_cast<double>(connection->getMaxQueueDataSize()),
+        {{"connection_uuid", connection->getUUIDStr()}, {"connection_name", connection->getName()}, {"metric_class", metric_class}}});
+      metrics.push_back({"queue_size", static_cast<double>(connection->getQueueSize()),
+        {{"connection_uuid", connection->getUUIDStr()}, {"connection_name", connection->getName()}, {"metric_class", metric_class}}});
+      metrics.push_back({"queue_size_max", static_cast<double>(connection->getMaxQueueSize()),
+        {{"connection_uuid", connection->getUUIDStr()}, {"connection_name", connection->getName()}, {"metric_class", metric_class}}});
+    }
+
+    return metrics;
+  }
+
+  virtual ~ConnectionStore() = default;
+
+ protected:
+  std::map<std::string, minifi::Connection*> connections_;

Review Comment:
   Consider using `std::unordered_map<utils::Identifier, Connection*>`.
   
   Untested pseudocode to help with `std::hash` specialization for `Identifier`:
   ```
   namespace std {
   template<>
   struct hash<Identifier> {
     size_t operator()(const Identifier& id) const noexcept {
       constexpr int slices = sizeof(Identifier) / sizeof(size_t);
       const auto combine = [](size_t& seed, size_t new_hash) {
         // from the boost hash_combine docs
         seed ^= new_hash + 0x9e3779b9 + (seed << 6) + (seed >> 2);
       };
       const auto get_slice = [](const Identifier& id, size_t idx) -> size_t {
         // Needs access to Identifier internals
         size_t result{};
         memcpy(&result, reinterpret_cast<const unsigned char*>(&id.data_) + idx * sizeof(size_t), sizeof(size_t));
         return result;
       };
       size_t hash = get_slice(id, 0);
       for (size_t i = 1; i < slices; ++i) {
         combine(hash, get_slice(id, i));
       };
       return hash;
     }
   };
   }  // namespace std
   
   
   



-- 
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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

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

   Removed the QueueMetrics class from default configuration in df6ae67603f4e75ecc45ab87d2acd2a401ff9cf6 due to the bug in C2 serialization described in https://issues.apache.org/jira/browse/MINIFICPP-1849


-- 
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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

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

   Added fix for [MINIFICPP-1849](https://issues.apache.org/jira/browse/MINIFICPP-1849) for serialization of empty QueueMetrics on C2 in 9c0f374283b8507f9940690547d267c188de0330. Readded QueueMetrics to the default configuration. 


-- 
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] adamdebreceni commented on a diff in pull request #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r892432982


##########
extensions/http-curl/tests/C2MetricsTest.cpp:
##########
@@ -0,0 +1,209 @@
+/**
+ *
+ * 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.
+ */
+
+#undef NDEBUG
+#include <string>
+#include <iostream>
+#include <filesystem>
+
+#include "TestBase.h"
+#include "HTTPIntegrationBase.h"
+#include "HTTPHandlers.h"
+#include "processors/TailFile.h"
+#include "state/ProcessorController.h"
+#include "utils/file/FileUtils.h"
+#include "utils/TestUtils.h"
+#include "processors/GetTCP.h"
+#include "utils/StringUtils.h"
+#include "utils/file/PathUtils.h"
+
+using namespace std::literals::chrono_literals;
+
+namespace org::apache::nifi::minifi::test {
+
+class VerifyC2Metrics : public VerifyC2Base {
+ public:
+  explicit VerifyC2Metrics(const std::atomic_bool& metrics_updated_successfully) : metrics_updated_successfully_(metrics_updated_successfully) {
+  }
+
+  void testSetup() override {
+    LogTestController::getInstance().setTrace<minifi::c2::C2Agent>();
+    LogTestController::getInstance().setTrace<minifi::c2::C2Client>();
+    LogTestController::getInstance().setDebug<minifi::c2::RESTSender>();
+    LogTestController::getInstance().setDebug<minifi::FlowController>();
+    LogTestController::getInstance().setOff<minifi::processors::GetTCP>();
+    VerifyC2Base::testSetup();
+  }
+
+  void runAssertions() override {
+    using org::apache::nifi::minifi::utils::verifyEventHappenedInPollTime;
+    assert(verifyEventHappenedInPollTime(40s, [&] { return metrics_updated_successfully_.load(); }, 1s));
+  }
+
+ private:
+  const std::atomic_bool& metrics_updated_successfully_;
+};
+
+class MetricsHandler: public HeartbeatHandler {
+ public:
+  explicit MetricsHandler(std::atomic_bool& metrics_updated_successfully, std::shared_ptr<minifi::Configure> configuration, const std::string& replacement_config_path)
+    : HeartbeatHandler(std::move(configuration)),
+      metrics_updated_successfully_(metrics_updated_successfully),
+      replacement_config_(getReplacementConfigAsJsonValue(replacement_config_path)) {
+  }
+
+  void handleHeartbeat(const rapidjson::Document& root, struct mg_connection* conn) override {
+    switch (test_state_) {
+      case TestState::VERIFY_INITIAL_METRICS: {
+        verifyMetrics(root);
+        sendEmptyHeartbeatResponse(conn);
+        break;
+      }
+      case TestState::SEND_NEW_CONFIG: {
+        sendHeartbeatResponse("UPDATE", "configuration", "889348", conn, {{"configuration_data", replacement_config_}});
+        test_state_ = TestState::VERIFY_UPDATED_METRICS;
+        break;
+      }
+      case TestState::VERIFY_UPDATED_METRICS: {
+        verifyUpdatedMetrics(root);
+        sendEmptyHeartbeatResponse(conn);
+        break;
+      }
+    }
+  }
+
+ private:
+  enum class TestState {
+    VERIFY_INITIAL_METRICS,
+    SEND_NEW_CONFIG,
+    VERIFY_UPDATED_METRICS
+  };
+
+  static void sendEmptyHeartbeatResponse(struct mg_connection* conn) {
+    mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: text/plain\r\nContent-Length: 0\r\nConnection: close\r\n\r\n");
+  }
+
+  void verifyMetrics(const rapidjson::Document& root) {
+    auto initial_metrics_verified =
+      root.HasMember("metrics") &&
+      root["metrics"].HasMember("RuntimeMetrics") &&
+      root["metrics"].HasMember("LoadMetrics") &&
+      root["metrics"].HasMember("ProcessorMetrics");
+    if (initial_metrics_verified) {
+      test_state_ = TestState::SEND_NEW_CONFIG;
+    }
+  }
+
+  void verifyUpdatedMetrics(const rapidjson::Document& root) {
+    auto updated_metrics_verified =
+      root.HasMember("metrics") &&
+      root["metrics"].HasMember("RuntimeMetrics") &&
+      root["metrics"].HasMember("LoadMetrics") &&
+      !root["metrics"].HasMember("ProcessorMetrics") &&
+      verifyUpdatedRuntimeMetrics(root["metrics"]["RuntimeMetrics"]) &&
+      verifyUpdatedLoadMetrics(root["metrics"]["LoadMetrics"]);
+
+    if (updated_metrics_verified) {
+      metrics_updated_successfully_ = true;
+    }
+  }
+
+  static bool verifyRuntimeMetrics(const rapidjson::Value& runtime_metrics) {
+    return runtime_metrics.HasMember("deviceInfo") &&
+      runtime_metrics.HasMember("flowInfo") &&
+      runtime_metrics["flowInfo"].HasMember("versionedFlowSnapshotURI") &&
+      runtime_metrics["flowInfo"].HasMember("queues") &&
+      runtime_metrics["flowInfo"].HasMember("components") &&
+      runtime_metrics["flowInfo"]["queues"].HasMember("2438e3c8-015a-1000-79ca-83af40ec1997") &&
+      runtime_metrics["flowInfo"]["components"].HasMember("FlowController") &&
+      runtime_metrics["flowInfo"]["components"].HasMember("GetTCP") &&
+      runtime_metrics["flowInfo"]["components"].HasMember("LogAttribute");
+  }

Review Comment:
   we have a `verifyJSON` test util, could that be used here? (maybe with some enhancements? e.g. `"__any__"` matching all values if we only care about field existence)



-- 
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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r898829922


##########
METRICS.md:
##########
@@ -0,0 +1,153 @@
+<!--
+  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.
+-->
+
+# Apache NiFi - MiNiFi - C++ Metrics Readme.
+
+
+This readme defines the metrics published by Apache NiFi. All options defined are located in minifi.properties.
+
+## Table of Contents
+
+- [Description](#description)
+- [Configuration](#configuration)
+- [Metrics](#metrics)
+
+## Description
+
+Apache NiFi MiNiFi C++ can communicate metrics about the agent's status, that can be a system level or component level metric.
+These metrics are exposed through the agent implemented metric publishers that can be configured in the minifi.properties.
+Aside from the publisher exposed metrics, metrics are also sent through C2 protocol of which there is more information in the
+[C2 documentation](C2.md#metrics).
+
+## Configuration
+
+To configure the a metrics publisher first we have to set which publisher class should be used:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.class=PrometheusMetricsPublisher
+
+Currently PrometheusMetricsPublisher is the only available publisher in MiNiFi C++ which publishes metrics to a Prometheus server.
+To use the publisher a port should also be configured where the metrics will be available to be scraped through:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.port=9936

Review Comment:
   As currently we only support having a single publisher configured at a time I wanted to reflect that in the configuration as well. If we changed it to `nifi.metrics.publisher.PrometheusMetricsPublisher.port` that would suggest that we can have multiple publishers in the `nifi.metrics.publisher.class` as well.
   
   The other option is to change the property to `nifi.metrics.publisher.classes` and prepare the code to handle multiple publishers in advance. That would make the configuration backwards compatible in the future, but would also introduce some unneeded complexity of parsing and handling the publishers at the moment.
   
   I would keep it like this for now and if we want to support exposing multiple publisher classes at the same time then we would change the configuration and the parsing accordingly. What do you think?



##########
extensions/prometheus/PublishedMetricGaugeCollection.cpp:
##########
@@ -0,0 +1,49 @@
+/**
+ * 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 "PublishedMetricGaugeCollection.h"
+
+#include <utility>
+#include <algorithm>
+
+#include "prometheus/client_metric.h"
+#include "state/PublishedMetricProvider.h"
+#include "range/v3/range/conversion.hpp"
+#include "range/v3/view/transform.hpp"
+
+namespace org::apache::nifi::minifi::extensions::prometheus {
+
+PublishedMetricGaugeCollection::PublishedMetricGaugeCollection(std::shared_ptr<state::PublishedMetricProvider> metric) : metric_{std::move(metric)} {
+}
+
+std::vector<::prometheus::MetricFamily> PublishedMetricGaugeCollection::Collect() const {
+  std::vector<::prometheus::MetricFamily> collection;
+  for (const auto& metric : metric_->calculateMetrics()) {
+    ::prometheus::ClientMetric client_metric;
+    client_metric.label = ranges::views::transform(metric.labels, [](auto&& kvp) { return ::prometheus::ClientMetric::Label{kvp.first, kvp.second}; })
+      | ranges::to<std::vector<::prometheus::ClientMetric::Label>>;
+    client_metric.gauge = ::prometheus::ClientMetric::Gauge{metric.value};
+    collection.push_back({
+      .name = metric.name,
+      .help = "",
+      .type = ::prometheus::MetricType::Gauge,
+      .metric = { client_metric }
+    });

Review Comment:
   Most guidelines discouraged prefixes for metric names and I didn't want to use it because Prometheus already adds a `job` label for the metric with the `minifi` value that can be filtered for. But I checked the [Prometheus best practices](https://prometheus.io/docs/practices/naming/) and it says that the metrics should have an application name as a prefix to mark the domain (or sometimes it's not an application name but something general like `http`). In our case using `minifi_` prefix is a good idea I added it in 96431d07a867a26723d9d6c2cf71edc1d1dd0b17



-- 
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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r907546457


##########
libminifi/include/utils/Id.h:
##########
@@ -132,3 +134,26 @@ class NonRepeatingStringGenerator {
 };
 
 }  // namespace org::apache::nifi::minifi::utils
+
+namespace std {
+template<>
+struct hash<org::apache::nifi::minifi::utils::Identifier> {
+  size_t operator()(const org::apache::nifi::minifi::utils::Identifier& id) const noexcept {
+    constexpr int slices = sizeof(org::apache::nifi::minifi::utils::Identifier) / sizeof(size_t);
+    const auto combine = [](size_t& seed, size_t new_hash) {
+      // from the boost hash_combine docs
+      seed ^= new_hash + 0x9e3779b9 + (seed << 6) + (seed >> 2);
+    };
+    const auto get_slice = [](const org::apache::nifi::minifi::utils::Identifier& id, size_t idx) -> size_t {
+      size_t result{};
+      memcpy(&result, reinterpret_cast<const unsigned char*>(&id.data_) + idx * sizeof(size_t), sizeof(size_t));
+      return result;
+    };
+    size_t hash = get_slice(id, 0);
+    for (size_t i = 1; i < slices; ++i) {
+      combine(hash, get_slice(id, i));
+    }
+    return hash;

Review Comment:
   Added assertion in 310e941b1e20d0bb2366d6d688fd5f89c718bd9b



-- 
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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r906190403


##########
libminifi/include/core/state/nodes/RepositoryMetrics.h:
##########
@@ -90,15 +87,18 @@ class RepositoryMetrics : public ResponseNode {
     return serialized;
   }
 
+  std::vector<PublishedMetric> calculateMetrics() override {
+    std::vector<PublishedMetric> metrics;
+    for (const auto& [_, repo] : repositories_) {
+      metrics.push_back({"is_running", (repo->isRunning() ? 1.0 : 0.0), {{"metric_class", getName()}, {"repository_name", repo->getName()}}});
+      metrics.push_back({"is_full", (repo->isFull() ? 1.0 : 0.0), {{"metric_class", getName()}, {"repository_name", repo->getName()}}});
+      metrics.push_back({"repository_size", static_cast<double>(repo->getRepoSize()), {{"metric_class", getName()}, {"repository_name", repo->getName()}}});
+    }
+    return metrics;
+  }

Review Comment:
   > Also moving the serialization and the metrics calculation to C2Client and the metrics publisher would require us to define parts of a newly introduced metric in 3 places, which would not be really intuitive.
   
   I was thinking about metric-independent formatting in C2Client and the prometheus extension, so new metrics require only 1 code change, not 2 or 3.
   
   I've added this to a followup refactoring Jira issue: https://issues.apache.org/jira/browse/MINIFICPP-1869



-- 
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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r908121606


##########
libminifi/src/FlowController.cpp:
##########
@@ -150,6 +154,7 @@ bool FlowController::applyConfiguration(const std::string &source, const std::st
     this->root_ = std::move(prevRoot);

Review Comment:
   I think in theory you are right, it could be unlocked if the exception happens after the unlock. Although the exception check is mostly meant for the loading of the new configuration, because after the unlock we only write some logs and set a few configuration values that shouldn't really throw any exceptions. Just in case I added the additional lock in 61844c0a17630875f7db41fcd329bedae7ef9df6



-- 
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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r905309695


##########
libminifi/include/core/state/nodes/RepositoryMetrics.h:
##########
@@ -90,15 +87,18 @@ class RepositoryMetrics : public ResponseNode {
     return serialized;
   }
 
+  std::vector<PublishedMetric> calculateMetrics() override {
+    std::vector<PublishedMetric> metrics;
+    for (const auto& [_, repo] : repositories_) {
+      metrics.push_back({"is_running", (repo->isRunning() ? 1.0 : 0.0), {{"metric_class", getName()}, {"repository_name", repo->getName()}}});
+      metrics.push_back({"is_full", (repo->isFull() ? 1.0 : 0.0), {{"metric_class", getName()}, {"repository_name", repo->getName()}}});
+      metrics.push_back({"repository_size", static_cast<double>(repo->getRepoSize()), {{"metric_class", getName()}, {"repository_name", repo->getName()}}});
+    }
+    return metrics;
+  }

Review Comment:
   Now each metric class needs to have two different member functions (serialize and calculateMetrics) that perform essentially the same task: collect metrics and format them in a specific way. I would prefer to separate the "collect metrics" part from the formatting, and have 1 "collect metrics" function here, and one formatting function in each of C2Client and somewhere in the prometheus extension. I didn't think about the details yet, only have this high level idea.



-- 
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] adamdebreceni commented on a diff in pull request #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r907461728


##########
libminifi/include/core/state/nodes/MetricsBase.h:
##########
@@ -108,27 +104,33 @@ class ObjectNode : public ResponseNode {
     nodes_.push_back(node);
   }
 
-  virtual std::string getName() const {
+  const std::vector<std::shared_ptr<ResponseNode>>& get_child_nodes() const {

Review Comment:
   I see that the casing is mixed in this class, IMO we should go with camelCase for new methods



-- 
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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r908160206


##########
libminifi/src/FlowController.cpp:
##########
@@ -150,6 +154,7 @@ bool FlowController::applyConfiguration(const std::string &source, const std::st
     this->root_ = std::move(prevRoot);

Review Comment:
   It could happen when `executeOnAllComponents` or `executeOnComponent` was called from a metric while updating that's why the `updating_` check was added there.



-- 
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] adamdebreceni commented on a diff in pull request #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r907525112


##########
libminifi/include/utils/Id.h:
##########
@@ -132,3 +134,26 @@ class NonRepeatingStringGenerator {
 };
 
 }  // namespace org::apache::nifi::minifi::utils
+
+namespace std {
+template<>
+struct hash<org::apache::nifi::minifi::utils::Identifier> {
+  size_t operator()(const org::apache::nifi::minifi::utils::Identifier& id) const noexcept {
+    constexpr int slices = sizeof(org::apache::nifi::minifi::utils::Identifier) / sizeof(size_t);
+    const auto combine = [](size_t& seed, size_t new_hash) {
+      // from the boost hash_combine docs
+      seed ^= new_hash + 0x9e3779b9 + (seed << 6) + (seed >> 2);
+    };
+    const auto get_slice = [](const org::apache::nifi::minifi::utils::Identifier& id, size_t idx) -> size_t {
+      size_t result{};
+      memcpy(&result, reinterpret_cast<const unsigned char*>(&id.data_) + idx * sizeof(size_t), sizeof(size_t));
+      return result;
+    };
+    size_t hash = get_slice(id, 0);
+    for (size_t i = 1; i < slices; ++i) {
+      combine(hash, get_slice(id, i));
+    }
+    return hash;

Review Comment:
   I was thinking about `sizeof(Identifier) % sizeof(size_t) == 0`, a simple `static_assert` will do



-- 
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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r893443384


##########
Windows.md:
##########
@@ -67,6 +67,7 @@ After the build directory it will take optional parameters modifying the CMake c
 | /L | Enables Linter |
 | /O | Enables OpenCV |
 | /PDH | Enables Performance Monitor |
+| /P | Enables Prometheus |

Review Comment:
   Good catch, fixed in 4f53b2c73daee9f416fab8ad038404064002d0e8



##########
extensions/http-curl/tests/C2MetricsTest.cpp:
##########
@@ -0,0 +1,209 @@
+/**
+ *
+ * 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.
+ */
+
+#undef NDEBUG
+#include <string>
+#include <iostream>
+#include <filesystem>
+
+#include "TestBase.h"
+#include "HTTPIntegrationBase.h"
+#include "HTTPHandlers.h"
+#include "processors/TailFile.h"
+#include "state/ProcessorController.h"
+#include "utils/file/FileUtils.h"
+#include "utils/TestUtils.h"
+#include "processors/GetTCP.h"
+#include "utils/StringUtils.h"
+#include "utils/file/PathUtils.h"
+
+using namespace std::literals::chrono_literals;
+
+namespace org::apache::nifi::minifi::test {
+
+class VerifyC2Metrics : public VerifyC2Base {
+ public:
+  explicit VerifyC2Metrics(const std::atomic_bool& metrics_updated_successfully) : metrics_updated_successfully_(metrics_updated_successfully) {
+  }
+
+  void testSetup() override {
+    LogTestController::getInstance().setTrace<minifi::c2::C2Agent>();
+    LogTestController::getInstance().setTrace<minifi::c2::C2Client>();
+    LogTestController::getInstance().setDebug<minifi::c2::RESTSender>();
+    LogTestController::getInstance().setDebug<minifi::FlowController>();
+    LogTestController::getInstance().setOff<minifi::processors::GetTCP>();
+    VerifyC2Base::testSetup();
+  }
+
+  void runAssertions() override {
+    using org::apache::nifi::minifi::utils::verifyEventHappenedInPollTime;
+    assert(verifyEventHappenedInPollTime(40s, [&] { return metrics_updated_successfully_.load(); }, 1s));
+  }
+
+ private:
+  const std::atomic_bool& metrics_updated_successfully_;
+};
+
+class MetricsHandler: public HeartbeatHandler {
+ public:
+  explicit MetricsHandler(std::atomic_bool& metrics_updated_successfully, std::shared_ptr<minifi::Configure> configuration, const std::string& replacement_config_path)
+    : HeartbeatHandler(std::move(configuration)),
+      metrics_updated_successfully_(metrics_updated_successfully),
+      replacement_config_(getReplacementConfigAsJsonValue(replacement_config_path)) {
+  }
+
+  void handleHeartbeat(const rapidjson::Document& root, struct mg_connection* conn) override {
+    switch (test_state_) {
+      case TestState::VERIFY_INITIAL_METRICS: {
+        verifyMetrics(root);
+        sendEmptyHeartbeatResponse(conn);
+        break;
+      }
+      case TestState::SEND_NEW_CONFIG: {
+        sendHeartbeatResponse("UPDATE", "configuration", "889348", conn, {{"configuration_data", replacement_config_}});
+        test_state_ = TestState::VERIFY_UPDATED_METRICS;
+        break;
+      }
+      case TestState::VERIFY_UPDATED_METRICS: {
+        verifyUpdatedMetrics(root);
+        sendEmptyHeartbeatResponse(conn);
+        break;
+      }
+    }
+  }
+
+ private:
+  enum class TestState {
+    VERIFY_INITIAL_METRICS,
+    SEND_NEW_CONFIG,
+    VERIFY_UPDATED_METRICS
+  };
+
+  static void sendEmptyHeartbeatResponse(struct mg_connection* conn) {
+    mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: text/plain\r\nContent-Length: 0\r\nConnection: close\r\n\r\n");
+  }
+
+  void verifyMetrics(const rapidjson::Document& root) {
+    auto initial_metrics_verified =
+      root.HasMember("metrics") &&
+      root["metrics"].HasMember("RuntimeMetrics") &&
+      root["metrics"].HasMember("LoadMetrics") &&
+      root["metrics"].HasMember("ProcessorMetrics");
+    if (initial_metrics_verified) {
+      test_state_ = TestState::SEND_NEW_CONFIG;
+    }
+  }
+
+  void verifyUpdatedMetrics(const rapidjson::Document& root) {
+    auto updated_metrics_verified =
+      root.HasMember("metrics") &&
+      root["metrics"].HasMember("RuntimeMetrics") &&
+      root["metrics"].HasMember("LoadMetrics") &&
+      !root["metrics"].HasMember("ProcessorMetrics") &&
+      verifyUpdatedRuntimeMetrics(root["metrics"]["RuntimeMetrics"]) &&
+      verifyUpdatedLoadMetrics(root["metrics"]["LoadMetrics"]);
+
+    if (updated_metrics_verified) {
+      metrics_updated_successfully_ = true;
+    }
+  }
+
+  static bool verifyRuntimeMetrics(const rapidjson::Value& runtime_metrics) {
+    return runtime_metrics.HasMember("deviceInfo") &&
+      runtime_metrics.HasMember("flowInfo") &&
+      runtime_metrics["flowInfo"].HasMember("versionedFlowSnapshotURI") &&
+      runtime_metrics["flowInfo"].HasMember("queues") &&
+      runtime_metrics["flowInfo"].HasMember("components") &&
+      runtime_metrics["flowInfo"]["queues"].HasMember("2438e3c8-015a-1000-79ca-83af40ec1997") &&
+      runtime_metrics["flowInfo"]["components"].HasMember("FlowController") &&
+      runtime_metrics["flowInfo"]["components"].HasMember("GetTCP") &&
+      runtime_metrics["flowInfo"]["components"].HasMember("LogAttribute");
+  }

Review Comment:
   I checked it, but I think in this case it's not really viable. There are a lot of environment or flow state specific metrics that cannot be checked for and exact expected json and I think it's better to check on these specific fields separately.



##########
extensions/prometheus/PrometheusExposerWrapper.cpp:
##########
@@ -0,0 +1,34 @@
+/**
+ * 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 "PrometheusExposerWrapper.h"
+
+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 %u", port);

Review Comment:
   Good point, updated in 4f53b2c73daee9f416fab8ad038404064002d0e8



##########
libminifi/include/core/state/nodes/QueueMetrics.h:
##########
@@ -94,15 +82,20 @@ class QueueMetrics : public ResponseNode {
     return serialized;
   }
 
- protected:
-  std::map<std::string, std::unique_ptr<minifi::Connection>> connections;
+  std::vector<PublishedMetric> calculateMetrics() override {
+    std::vector<PublishedMetric> metrics;
+    for (const auto& [_, connection] : connections_) {
+      metrics.push_back({"queue_data_size", static_cast<double>(connection->getQueueDataSize()),
+        {{"connection_uuid", connection->getUUIDStr()}, {"connection_name", connection->getName()}, {"metric_class", getName()}}});
+      metrics.push_back({"queue_data_size_max", static_cast<double>(connection->getMaxQueueDataSize()),
+        {{"connection_uuid", connection->getUUIDStr()}, {"connection_name", connection->getName()}, {"metric_class", getName()}}});
+      metrics.push_back({"queue_size", static_cast<double>(connection->getQueueSize()),
+        {{"connection_uuid", connection->getUUIDStr()}, {"connection_name", connection->getName()}, {"metric_class", getName()}}});
+      metrics.push_back({"queue_size_max", static_cast<double>(connection->getMaxQueueSize()),
+        {{"connection_uuid", connection->getUUIDStr()}, {"connection_name", connection->getName()}, {"metric_class", getName()}}});
+    }
+    return metrics;

Review Comment:
   I moved the connection metric calculation to the ConnectionStore base class which returns a PublishedMetrics collection. Currently it has the name, value and labels structure as a data format for providing metrics. We shall see if that holds up to be generic enough when another metric publisher is implemented or it needs to be changed.



##########
libminifi/src/core/state/nodes/QueueMetrics.cpp:
##########
@@ -0,0 +1,27 @@
+/**
+ *
+ * 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 "core/state/nodes/QueueMetrics.h"
+#include "core/Resource.h"
+
+namespace org::apache::nifi::minifi::state::response {
+
+REGISTER_RESOURCE(QueueMetrics, "Node part of an AST that defines queue metric information");

Review Comment:
   I think it's a legacy term used in all the metric node descriptions, not sure why, as in C2 the metric nodes are represented in kind of a tree format, but not an AST. I changed it to be a more generic "Metric node" in the descriptions in 4f53b2c73daee9f416fab8ad038404064002d0e8.



##########
docker/test/integration/minifi/core/DockerTestCluster.py:
##########
@@ -283,3 +282,79 @@ def write_content_to_container(self, content, dst):
                 tar.addfile(info, io.BytesIO(content.encode('utf-8')))
             with open(os.path.join(td, 'content.tar'), 'rb') as data:
                 return container.put_archive(os.path.dirname(dst_path), data.read())
+
+    def check_metric_class_on_prometheus(self, metric_class, timeout_seconds):
+        start_time = time.perf_counter()
+        while (time.perf_counter() - start_time) < timeout_seconds:
+            if self.verify_metric_class(metric_class):
+                return True
+            time.sleep(1)
+        return False

Review Comment:
   Good point, changed it in 4f53b2c73daee9f416fab8ad038404064002d0e8



##########
libminifi/include/FlowController.h:
##########
@@ -252,8 +255,7 @@ class FlowController : public core::controller::ForwardingControllerServiceProvi
   // Thread pool for schedulers
   utils::ThreadPool<utils::TaskRescheduleInfo> thread_pool_;
   std::map<utils::Identifier, std::unique_ptr<state::ProcessorController>> processor_to_controller_;
+  std::unique_ptr<state::MetricsPublisher> metrics_publisher_;

Review Comment:
   I agree, it should not depend on it, and I started out with that concept at the beginning, but unfortunately I had a few major problems with the current design in the FlowContrroller, C2Client and the metric ResponseNodes.
   
   1. There are some metric nodes that depend on parts of the FlowController like the AgentNode which depends on the controller service provider to be initialized when loading that node. This requires that the ResponseNodeLoader should only be initialized after the FlowController initialized its `controller_service_provider_impl_` member and only after that we can initialize the metrics node publisher to make sure we can load any of the configured metric nodes.
   2. In case of a flow update we need to make sure to remove all the metric nodes before applying the flow config, to avoid problems when a metric is collected on another thread and tries to query non-existing connections or processgroups. After the update these metric nodes need to be repopulated and as the FlowController controls this configuration update and the new flow configuration, we unfortunately have to depend on it to provide the new flow and take care of the thread synchronization while updating.
   
   These dependencies are not ideal at all and should be changed in the future, but looks to be a cumbersome change to figure out how these dependencies should be fixed and managed later in the future. I would propose a separate jira ticket for a future PR that resolves these issue, if that okay?



-- 
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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r905305999


##########
libminifi/include/properties/Configuration.h:
##########
@@ -156,12 +156,17 @@ class Configuration : public Properties {
 
   static constexpr const char *nifi_asset_directory = "nifi.asset.directory";
 
+  // Metrics publisher options
+  static constexpr const char *nifi_metrics_publisher_class = "nifi.metrics.publisher.class";
+  static constexpr const char *nifi_metrics_publisher_prometheus_metrics_publisher_port = "nifi.metrics.publisher.PrometheusMetricsPublisher.port";

Review Comment:
   Fair point. Let's leave it as is, and maybe think about it later, if it starts to become a problem.



-- 
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] adamdebreceni commented on a diff in pull request #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r907497990


##########
libminifi/include/utils/Id.h:
##########
@@ -132,3 +134,26 @@ class NonRepeatingStringGenerator {
 };
 
 }  // namespace org::apache::nifi::minifi::utils
+
+namespace std {
+template<>
+struct hash<org::apache::nifi::minifi::utils::Identifier> {
+  size_t operator()(const org::apache::nifi::minifi::utils::Identifier& id) const noexcept {
+    constexpr int slices = sizeof(org::apache::nifi::minifi::utils::Identifier) / sizeof(size_t);
+    const auto combine = [](size_t& seed, size_t new_hash) {
+      // from the boost hash_combine docs
+      seed ^= new_hash + 0x9e3779b9 + (seed << 6) + (seed >> 2);
+    };
+    const auto get_slice = [](const org::apache::nifi::minifi::utils::Identifier& id, size_t idx) -> size_t {
+      size_t result{};
+      memcpy(&result, reinterpret_cast<const unsigned char*>(&id.data_) + idx * sizeof(size_t), sizeof(size_t));
+      return result;
+    };
+    size_t hash = get_slice(id, 0);
+    for (size_t i = 1; i < slices; ++i) {
+      combine(hash, get_slice(id, i));
+    }
+    return hash;

Review Comment:
   we could use `gsl::span` here, it would also verify convertibility
   ```
       for (size_t slice : gsl::span(id.data_).as_span<const size_t>()) {
         combine(hash, slice);
       }
   ```



-- 
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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r908140314


##########
libminifi/src/FlowController.cpp:
##########
@@ -150,6 +154,7 @@ bool FlowController::applyConfiguration(const std::string &source, const std::st
     this->root_ = std::move(prevRoot);

Review Comment:
   As far as I remember it could cause a deadlock when the updating_ flag is checked from another thread and it's already set to false trying to acquire the lock afterwards, but it's still locked, so we have to unlock it before setting the update_ flag back to false.



-- 
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] adamdebreceni commented on a diff in pull request #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r907443301


##########
extensions/prometheus/PublishedMetricGaugeCollection.cpp:
##########
@@ -0,0 +1,49 @@
+/**
+ * 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 "PublishedMetricGaugeCollection.h"
+
+#include <utility>
+#include <algorithm>
+
+#include "prometheus/client_metric.h"
+#include "state/PublishedMetricProvider.h"
+#include "range/v3/range/conversion.hpp"
+#include "range/v3/view/transform.hpp"
+
+namespace org::apache::nifi::minifi::extensions::prometheus {
+
+PublishedMetricGaugeCollection::PublishedMetricGaugeCollection(std::shared_ptr<state::PublishedMetricProvider> metric) : metric_{std::move(metric)} {
+}
+
+std::vector<::prometheus::MetricFamily> PublishedMetricGaugeCollection::Collect() const {
+  std::vector<::prometheus::MetricFamily> collection;
+  for (const auto& metric : metric_->calculateMetrics()) {
+    ::prometheus::ClientMetric client_metric;
+    client_metric.label = ranges::views::transform(metric.labels, [](auto&& kvp) { return ::prometheus::ClientMetric::Label{kvp.first, kvp.second}; })
+      | ranges::to<std::vector<::prometheus::ClientMetric::Label>>;
+    client_metric.gauge = ::prometheus::ClientMetric::Gauge{metric.value};
+    collection.push_back({
+      .name = "minifi_" + metric.name,
+      .help = "",
+      .type = ::prometheus::MetricType::Gauge,
+      .metric = { client_metric }

Review Comment:
   we could move `client_metric` 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] adamdebreceni commented on a diff in pull request #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r892361490


##########
Windows.md:
##########
@@ -67,6 +67,7 @@ After the build directory it will take optional parameters modifying the CMake c
 | /L | Enables Linter |
 | /O | Enables OpenCV |
 | /PDH | Enables Performance Monitor |
+| /P | Enables Prometheus |

Review Comment:
   shouldn't this be `PR`?



-- 
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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r899183077


##########
METRICS.md:
##########
@@ -0,0 +1,153 @@
+<!--
+  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.
+-->
+
+# Apache NiFi - MiNiFi - C++ Metrics Readme.
+
+
+This readme defines the metrics published by Apache NiFi. All options defined are located in minifi.properties.
+
+## Table of Contents
+
+- [Description](#description)
+- [Configuration](#configuration)
+- [Metrics](#metrics)
+
+## Description
+
+Apache NiFi MiNiFi C++ can communicate metrics about the agent's status, that can be a system level or component level metric.
+These metrics are exposed through the agent implemented metric publishers that can be configured in the minifi.properties.
+Aside from the publisher exposed metrics, metrics are also sent through C2 protocol of which there is more information in the
+[C2 documentation](C2.md#metrics).
+
+## Configuration
+
+To configure the a metrics publisher first we have to set which publisher class should be used:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.class=PrometheusMetricsPublisher
+
+Currently PrometheusMetricsPublisher is the only available publisher in MiNiFi C++ which publishes metrics to a Prometheus server.
+To use the publisher a port should also be configured where the metrics will be available to be scraped through:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.port=9936

Review Comment:
   Even if there can only be one publisher, I want to prepare the configuration to future publishers that don't use exactly one TCP or UDP port. It would be a shame if the next implementation of the interface had to break the config of Prometheus+MiNiFi users.
   
   I'm fine with the other option being `nifi.metrics.publisher.class` and possibly still supporting a comma-separated list of classes later. It can stay as zero or one classes now, and change later, since that change doesn't break backwards compatibility. I don't think that a differently named property and a string split by commas is too much extra complexity in exchange for better future proofing.



-- 
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] adamdebreceni commented on a diff in pull request #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r892432982


##########
extensions/http-curl/tests/C2MetricsTest.cpp:
##########
@@ -0,0 +1,209 @@
+/**
+ *
+ * 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.
+ */
+
+#undef NDEBUG
+#include <string>
+#include <iostream>
+#include <filesystem>
+
+#include "TestBase.h"
+#include "HTTPIntegrationBase.h"
+#include "HTTPHandlers.h"
+#include "processors/TailFile.h"
+#include "state/ProcessorController.h"
+#include "utils/file/FileUtils.h"
+#include "utils/TestUtils.h"
+#include "processors/GetTCP.h"
+#include "utils/StringUtils.h"
+#include "utils/file/PathUtils.h"
+
+using namespace std::literals::chrono_literals;
+
+namespace org::apache::nifi::minifi::test {
+
+class VerifyC2Metrics : public VerifyC2Base {
+ public:
+  explicit VerifyC2Metrics(const std::atomic_bool& metrics_updated_successfully) : metrics_updated_successfully_(metrics_updated_successfully) {
+  }
+
+  void testSetup() override {
+    LogTestController::getInstance().setTrace<minifi::c2::C2Agent>();
+    LogTestController::getInstance().setTrace<minifi::c2::C2Client>();
+    LogTestController::getInstance().setDebug<minifi::c2::RESTSender>();
+    LogTestController::getInstance().setDebug<minifi::FlowController>();
+    LogTestController::getInstance().setOff<minifi::processors::GetTCP>();
+    VerifyC2Base::testSetup();
+  }
+
+  void runAssertions() override {
+    using org::apache::nifi::minifi::utils::verifyEventHappenedInPollTime;
+    assert(verifyEventHappenedInPollTime(40s, [&] { return metrics_updated_successfully_.load(); }, 1s));
+  }
+
+ private:
+  const std::atomic_bool& metrics_updated_successfully_;
+};
+
+class MetricsHandler: public HeartbeatHandler {
+ public:
+  explicit MetricsHandler(std::atomic_bool& metrics_updated_successfully, std::shared_ptr<minifi::Configure> configuration, const std::string& replacement_config_path)
+    : HeartbeatHandler(std::move(configuration)),
+      metrics_updated_successfully_(metrics_updated_successfully),
+      replacement_config_(getReplacementConfigAsJsonValue(replacement_config_path)) {
+  }
+
+  void handleHeartbeat(const rapidjson::Document& root, struct mg_connection* conn) override {
+    switch (test_state_) {
+      case TestState::VERIFY_INITIAL_METRICS: {
+        verifyMetrics(root);
+        sendEmptyHeartbeatResponse(conn);
+        break;
+      }
+      case TestState::SEND_NEW_CONFIG: {
+        sendHeartbeatResponse("UPDATE", "configuration", "889348", conn, {{"configuration_data", replacement_config_}});
+        test_state_ = TestState::VERIFY_UPDATED_METRICS;
+        break;
+      }
+      case TestState::VERIFY_UPDATED_METRICS: {
+        verifyUpdatedMetrics(root);
+        sendEmptyHeartbeatResponse(conn);
+        break;
+      }
+    }
+  }
+
+ private:
+  enum class TestState {
+    VERIFY_INITIAL_METRICS,
+    SEND_NEW_CONFIG,
+    VERIFY_UPDATED_METRICS
+  };
+
+  static void sendEmptyHeartbeatResponse(struct mg_connection* conn) {
+    mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: text/plain\r\nContent-Length: 0\r\nConnection: close\r\n\r\n");
+  }
+
+  void verifyMetrics(const rapidjson::Document& root) {
+    auto initial_metrics_verified =
+      root.HasMember("metrics") &&
+      root["metrics"].HasMember("RuntimeMetrics") &&
+      root["metrics"].HasMember("LoadMetrics") &&
+      root["metrics"].HasMember("ProcessorMetrics");
+    if (initial_metrics_verified) {
+      test_state_ = TestState::SEND_NEW_CONFIG;
+    }
+  }
+
+  void verifyUpdatedMetrics(const rapidjson::Document& root) {
+    auto updated_metrics_verified =
+      root.HasMember("metrics") &&
+      root["metrics"].HasMember("RuntimeMetrics") &&
+      root["metrics"].HasMember("LoadMetrics") &&
+      !root["metrics"].HasMember("ProcessorMetrics") &&
+      verifyUpdatedRuntimeMetrics(root["metrics"]["RuntimeMetrics"]) &&
+      verifyUpdatedLoadMetrics(root["metrics"]["LoadMetrics"]);
+
+    if (updated_metrics_verified) {
+      metrics_updated_successfully_ = true;
+    }
+  }
+
+  static bool verifyRuntimeMetrics(const rapidjson::Value& runtime_metrics) {
+    return runtime_metrics.HasMember("deviceInfo") &&
+      runtime_metrics.HasMember("flowInfo") &&
+      runtime_metrics["flowInfo"].HasMember("versionedFlowSnapshotURI") &&
+      runtime_metrics["flowInfo"].HasMember("queues") &&
+      runtime_metrics["flowInfo"].HasMember("components") &&
+      runtime_metrics["flowInfo"]["queues"].HasMember("2438e3c8-015a-1000-79ca-83af40ec1997") &&
+      runtime_metrics["flowInfo"]["components"].HasMember("FlowController") &&
+      runtime_metrics["flowInfo"]["components"].HasMember("GetTCP") &&
+      runtime_metrics["flowInfo"]["components"].HasMember("LogAttribute");
+  }

Review Comment:
   we have a `verifyJSON` test util, could that be used here? (maybe with some enhancements? e.g. "__any__" matching all values if we only care about field existence)



-- 
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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r899503240


##########
METRICS.md:
##########
@@ -0,0 +1,153 @@
+<!--
+  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.
+-->
+
+# Apache NiFi - MiNiFi - C++ Metrics Readme.
+
+
+This readme defines the metrics published by Apache NiFi. All options defined are located in minifi.properties.
+
+## Table of Contents
+
+- [Description](#description)
+- [Configuration](#configuration)
+- [Metrics](#metrics)
+
+## Description
+
+Apache NiFi MiNiFi C++ can communicate metrics about the agent's status, that can be a system level or component level metric.
+These metrics are exposed through the agent implemented metric publishers that can be configured in the minifi.properties.
+Aside from the publisher exposed metrics, metrics are also sent through C2 protocol of which there is more information in the
+[C2 documentation](C2.md#metrics).
+
+## Configuration
+
+To configure the a metrics publisher first we have to set which publisher class should be used:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.class=PrometheusMetricsPublisher
+
+Currently PrometheusMetricsPublisher is the only available publisher in MiNiFi C++ which publishes metrics to a Prometheus server.
+To use the publisher a port should also be configured where the metrics will be available to be scraped through:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.port=9936

Review Comment:
   It begs the question, which of those values belong to which publisher? What if a publisher doesn't use the network, or not through TCP/UDP? Or it periodically posts via HTTP, in which the URL already has the port?
   
   We could declare a global port setting and have the publishers read from it if they want to, but it fundamentally belongs to the component that actually uses that port, if they need such a setting.



-- 
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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r907513027


##########
libminifi/include/utils/Id.h:
##########
@@ -132,3 +134,26 @@ class NonRepeatingStringGenerator {
 };
 
 }  // namespace org::apache::nifi::minifi::utils
+
+namespace std {
+template<>
+struct hash<org::apache::nifi::minifi::utils::Identifier> {
+  size_t operator()(const org::apache::nifi::minifi::utils::Identifier& id) const noexcept {
+    constexpr int slices = sizeof(org::apache::nifi::minifi::utils::Identifier) / sizeof(size_t);
+    const auto combine = [](size_t& seed, size_t new_hash) {
+      // from the boost hash_combine docs
+      seed ^= new_hash + 0x9e3779b9 + (seed << 6) + (seed >> 2);
+    };
+    const auto get_slice = [](const org::apache::nifi::minifi::utils::Identifier& id, size_t idx) -> size_t {
+      size_t result{};
+      memcpy(&result, reinterpret_cast<const unsigned char*>(&id.data_) + idx * sizeof(size_t), sizeof(size_t));
+      return result;
+    };
+    size_t hash = get_slice(id, 0);
+    for (size_t i = 1; i < slices; ++i) {
+      combine(hash, get_slice(id, i));
+    }
+    return hash;

Review Comment:
   I don't understand what kind of convertibility you mean here. Using `gsl::span::as_span` is just a reinterpret_cast internally, which usually results in UB because of strict aliasing violation. It's fine to use it to view the bytes of the object representation, and a handful of other cases, but usually `memcpy` is the safer option.
   
   Another alternative that I didn't think of at the time was to `const auto chunks = std::bit_cast<std::array<size_t, slices>>(id.data_);`, and iterate over the resulting array. This uses `memcpy` internally 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] szaszm commented on a diff in pull request #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r907513027


##########
libminifi/include/utils/Id.h:
##########
@@ -132,3 +134,26 @@ class NonRepeatingStringGenerator {
 };
 
 }  // namespace org::apache::nifi::minifi::utils
+
+namespace std {
+template<>
+struct hash<org::apache::nifi::minifi::utils::Identifier> {
+  size_t operator()(const org::apache::nifi::minifi::utils::Identifier& id) const noexcept {
+    constexpr int slices = sizeof(org::apache::nifi::minifi::utils::Identifier) / sizeof(size_t);
+    const auto combine = [](size_t& seed, size_t new_hash) {
+      // from the boost hash_combine docs
+      seed ^= new_hash + 0x9e3779b9 + (seed << 6) + (seed >> 2);
+    };
+    const auto get_slice = [](const org::apache::nifi::minifi::utils::Identifier& id, size_t idx) -> size_t {
+      size_t result{};
+      memcpy(&result, reinterpret_cast<const unsigned char*>(&id.data_) + idx * sizeof(size_t), sizeof(size_t));
+      return result;
+    };
+    size_t hash = get_slice(id, 0);
+    for (size_t i = 1; i < slices; ++i) {
+      combine(hash, get_slice(id, i));
+    }
+    return hash;

Review Comment:
   I don't understand what kind of convertibility you mean here. Using `gsl::span::as_span` is just a reinterpret_cast internally, which usually results in UB because of strict aliasing violation. It's fine to use it to view the bytes of the object representation, and a handful of other cases, but usually `memcpy` is the safer option.
   
   Another alternative that I didn't think of at the time was to `const auto chunks = std::bit_cast<std::array<size_t, slices>>(id.data_);`, and accumulate the resulting array. This uses `memcpy` internally 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] szaszm commented on a diff in pull request #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r907513027


##########
libminifi/include/utils/Id.h:
##########
@@ -132,3 +134,26 @@ class NonRepeatingStringGenerator {
 };
 
 }  // namespace org::apache::nifi::minifi::utils
+
+namespace std {
+template<>
+struct hash<org::apache::nifi::minifi::utils::Identifier> {
+  size_t operator()(const org::apache::nifi::minifi::utils::Identifier& id) const noexcept {
+    constexpr int slices = sizeof(org::apache::nifi::minifi::utils::Identifier) / sizeof(size_t);
+    const auto combine = [](size_t& seed, size_t new_hash) {
+      // from the boost hash_combine docs
+      seed ^= new_hash + 0x9e3779b9 + (seed << 6) + (seed >> 2);
+    };
+    const auto get_slice = [](const org::apache::nifi::minifi::utils::Identifier& id, size_t idx) -> size_t {
+      size_t result{};
+      memcpy(&result, reinterpret_cast<const unsigned char*>(&id.data_) + idx * sizeof(size_t), sizeof(size_t));
+      return result;
+    };
+    size_t hash = get_slice(id, 0);
+    for (size_t i = 1; i < slices; ++i) {
+      combine(hash, get_slice(id, i));
+    }
+    return hash;

Review Comment:
   I don't understand what kind of convertibility you mean here. Using `gsl::span::as_span` is just a reinterpret_cast internally, which usually results in UB because of strict aliasing violation. It's fine to use it to view the bytes of the object representation, and a handful of other cases, but usually `memcpy` is the safer option.
   
   Another alternative that I didn't think of at the time was to `const auto chunks = std::bit_cast<std::array<size_t, slices>>(id.data_);`, and accumulate/combine/left-fold the resulting array. This uses `memcpy` internally 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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r908130668


##########
libminifi/src/FlowController.cpp:
##########
@@ -150,6 +154,7 @@ bool FlowController::applyConfiguration(const std::string &source, const std::st
     this->root_ = std::move(prevRoot);

Review Comment:
   After realizing that it could throw an exception if we lock multiple times I reverted this change. It shouldn't throw when it is unlocked so it should be okay.



-- 
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] adamdebreceni commented on a diff in pull request #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r908158817


##########
libminifi/src/FlowController.cpp:
##########
@@ -150,6 +154,7 @@ bool FlowController::applyConfiguration(const std::string &source, const std::st
     this->root_ = std::move(prevRoot);

Review Comment:
   we should restructure this method to avoid manual locking, the whole lock protected region could get its own block, we should move the logging block out of the try block to the very end of this method
   
   ```
     updating_ = true;
   
     {
       std::lock_guard<std::recursive_mutex> flow_lock(mutex_);
       stop();
       unload();
       controller_map_->clear();
       clearResponseNodes();
       if (metrics_publisher_) {
         metrics_publisher_->clearMetricNodes();
       }
       auto prevRoot = std::move(this->root_);
       this->root_ = std::move(newRoot);
       processor_to_controller_.clear();
       initialized_ = false;
       bool started = false;
       try {
         load(std::move(root_), true);
         flow_update_ = true;
         started = start() == 0;
       } catch (...) {
         this->root_ = std::move(prevRoot);
         load(std::move(this->root_), true);
         flow_update_ = true;
       }
     }
   
     updating_ = false;
   
     if (started) {
       auto flowVersion = flow_configuration_->getFlowVersion();
       if (flowVersion) {
         logger_->log_debug("Setting flow id to %s", flowVersion->getFlowId());
         configuration_->set(Configure::nifi_c2_flow_id, flowVersion->getFlowId());
         configuration_->set(Configure::nifi_c2_flow_url, flowVersion->getFlowIdentifier()->getRegistryUrl());
       } else {
         logger_->log_debug("Invalid flow version, not setting");
       }
     }
   
     return started;
   ```



-- 
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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r898166387


##########
METRICS.md:
##########
@@ -0,0 +1,153 @@
+<!--
+  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.
+-->
+
+# Apache NiFi - MiNiFi - C++ Metrics Readme.
+
+
+This readme defines the metrics published by Apache NiFi. All options defined are located in minifi.properties.
+
+## Table of Contents
+
+- [Description](#description)
+- [Configuration](#configuration)
+- [Metrics](#metrics)
+
+## Description
+
+Apache NiFi MiNiFi C++ can communicate metrics about the agent's status, that can be a system level or component level metric.
+These metrics are exposed through the agent implemented metric publishers that can be configured in the minifi.properties.
+Aside from the publisher exposed metrics, metrics are also sent through C2 protocol of which there is more information in the
+[C2 documentation](C2.md#metrics).
+
+## Configuration
+
+To configure the a metrics publisher first we have to set which publisher class should be used:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.class=PrometheusMetricsPublisher
+
+Currently PrometheusMetricsPublisher is the only available publisher in MiNiFi C++ which publishes metrics to a Prometheus server.
+To use the publisher a port should also be configured where the metrics will be available to be scraped through:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.port=9936

Review Comment:
   This port should be a property of PrometheusMetricsPublisher, not all publishers in general.



##########
extensions/prometheus/PublishedMetricGaugeCollection.cpp:
##########
@@ -0,0 +1,49 @@
+/**
+ * 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 "PublishedMetricGaugeCollection.h"
+
+#include <utility>
+#include <algorithm>
+
+#include "prometheus/client_metric.h"
+#include "state/PublishedMetricProvider.h"
+#include "range/v3/range/conversion.hpp"
+#include "range/v3/view/transform.hpp"
+
+namespace org::apache::nifi::minifi::extensions::prometheus {
+
+PublishedMetricGaugeCollection::PublishedMetricGaugeCollection(std::shared_ptr<state::PublishedMetricProvider> metric) : metric_{std::move(metric)} {
+}
+
+std::vector<::prometheus::MetricFamily> PublishedMetricGaugeCollection::Collect() const {
+  std::vector<::prometheus::MetricFamily> collection;
+  for (const auto& metric : metric_->calculateMetrics()) {
+    ::prometheus::ClientMetric client_metric;
+    client_metric.label = ranges::views::transform(metric.labels, [](auto&& kvp) { return ::prometheus::ClientMetric::Label{kvp.first, kvp.second}; })
+      | ranges::to<std::vector<::prometheus::ClientMetric::Label>>;
+    client_metric.gauge = ::prometheus::ClientMetric::Gauge{metric.value};
+    collection.push_back({
+      .name = metric.name,
+      .help = "",
+      .type = ::prometheus::MetricType::Gauge,
+      .metric = { client_metric }
+    });

Review Comment:
   I think it would help if metric names were prefixed with "minifi_" before serving the data to prometheus. It just makes finding them much easier if multiple programs with different metrics are used.
   
   ![image](https://user-images.githubusercontent.com/1170582/173875908-d120e0f3-4aeb-4da9-a478-a1442ffebebc.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.

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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r899204423


##########
METRICS.md:
##########
@@ -0,0 +1,153 @@
+<!--
+  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.
+-->
+
+# Apache NiFi - MiNiFi - C++ Metrics Readme.
+
+
+This readme defines the metrics published by Apache NiFi. All options defined are located in minifi.properties.
+
+## Table of Contents
+
+- [Description](#description)
+- [Configuration](#configuration)
+- [Metrics](#metrics)
+
+## Description
+
+Apache NiFi MiNiFi C++ can communicate metrics about the agent's status, that can be a system level or component level metric.
+These metrics are exposed through the agent implemented metric publishers that can be configured in the minifi.properties.
+Aside from the publisher exposed metrics, metrics are also sent through C2 protocol of which there is more information in the
+[C2 documentation](C2.md#metrics).
+
+## Configuration
+
+To configure the a metrics publisher first we have to set which publisher class should be used:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.class=PrometheusMetricsPublisher
+
+Currently PrometheusMetricsPublisher is the only available publisher in MiNiFi C++ which publishes metrics to a Prometheus server.
+To use the publisher a port should also be configured where the metrics will be available to be scraped through:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.port=9936

Review Comment:
   If our goal is to support more ports for future publishers, wouldn't that be easier to use comma separated values for `nifi.metrics.publisher.port` for those publishers?



-- 
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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r899526210


##########
METRICS.md:
##########
@@ -0,0 +1,153 @@
+<!--
+  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.
+-->
+
+# Apache NiFi - MiNiFi - C++ Metrics Readme.
+
+
+This readme defines the metrics published by Apache NiFi. All options defined are located in minifi.properties.
+
+## Table of Contents
+
+- [Description](#description)
+- [Configuration](#configuration)
+- [Metrics](#metrics)
+
+## Description
+
+Apache NiFi MiNiFi C++ can communicate metrics about the agent's status, that can be a system level or component level metric.
+These metrics are exposed through the agent implemented metric publishers that can be configured in the minifi.properties.
+Aside from the publisher exposed metrics, metrics are also sent through C2 protocol of which there is more information in the
+[C2 documentation](C2.md#metrics).
+
+## Configuration
+
+To configure the a metrics publisher first we have to set which publisher class should be used:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.class=PrometheusMetricsPublisher
+
+Currently PrometheusMetricsPublisher is the only available publisher in MiNiFi C++ which publishes metrics to a Prometheus server.
+To use the publisher a port should also be configured where the metrics will be available to be scraped through:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.port=9936

Review Comment:
   What I meant that if we stay with supporting one publisher at a time for now, then the port option can have multiple ports separated by commas, or if not needed at all it can be disregarded while parsing the config.
   
   I see what you mean now, that the port may not always be a global option for all publishers so it should belong under `PrometheusMetricsPublisher.port` option, it makes more sense to change this with this in mind. 



-- 
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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r907494752


##########
extensions/prometheus/PublishedMetricGaugeCollection.cpp:
##########
@@ -0,0 +1,49 @@
+/**
+ * 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 "PublishedMetricGaugeCollection.h"
+
+#include <utility>
+#include <algorithm>
+
+#include "prometheus/client_metric.h"
+#include "state/PublishedMetricProvider.h"
+#include "range/v3/range/conversion.hpp"
+#include "range/v3/view/transform.hpp"
+
+namespace org::apache::nifi::minifi::extensions::prometheus {
+
+PublishedMetricGaugeCollection::PublishedMetricGaugeCollection(std::shared_ptr<state::PublishedMetricProvider> metric) : metric_{std::move(metric)} {
+}
+
+std::vector<::prometheus::MetricFamily> PublishedMetricGaugeCollection::Collect() const {
+  std::vector<::prometheus::MetricFamily> collection;
+  for (const auto& metric : metric_->calculateMetrics()) {
+    ::prometheus::ClientMetric client_metric;
+    client_metric.label = ranges::views::transform(metric.labels, [](auto&& kvp) { return ::prometheus::ClientMetric::Label{kvp.first, kvp.second}; })
+      | ranges::to<std::vector<::prometheus::ClientMetric::Label>>;
+    client_metric.gauge = ::prometheus::ClientMetric::Gauge{metric.value};
+    collection.push_back({
+      .name = "minifi_" + metric.name,
+      .help = "",
+      .type = ::prometheus::MetricType::Gauge,
+      .metric = { client_metric }

Review Comment:
   Updated in 40b469f11033f229a9237afd0b3ba397cb2e316d



##########
libminifi/include/core/state/nodes/MetricsBase.h:
##########
@@ -108,27 +104,33 @@ class ObjectNode : public ResponseNode {
     nodes_.push_back(node);
   }
 
-  virtual std::string getName() const {
+  const std::vector<std::shared_ptr<ResponseNode>>& get_child_nodes() const {

Review Comment:
   Removed `get_child_nodes` in 40b469f11033f229a9237afd0b3ba397cb2e316d because it is not used anymore.



-- 
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] adamdebreceni commented on a diff in pull request #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r907497990


##########
libminifi/include/utils/Id.h:
##########
@@ -132,3 +134,26 @@ class NonRepeatingStringGenerator {
 };
 
 }  // namespace org::apache::nifi::minifi::utils
+
+namespace std {
+template<>
+struct hash<org::apache::nifi::minifi::utils::Identifier> {
+  size_t operator()(const org::apache::nifi::minifi::utils::Identifier& id) const noexcept {
+    constexpr int slices = sizeof(org::apache::nifi::minifi::utils::Identifier) / sizeof(size_t);
+    const auto combine = [](size_t& seed, size_t new_hash) {
+      // from the boost hash_combine docs
+      seed ^= new_hash + 0x9e3779b9 + (seed << 6) + (seed >> 2);
+    };
+    const auto get_slice = [](const org::apache::nifi::minifi::utils::Identifier& id, size_t idx) -> size_t {
+      size_t result{};
+      memcpy(&result, reinterpret_cast<const unsigned char*>(&id.data_) + idx * sizeof(size_t), sizeof(size_t));
+      return result;
+    };
+    size_t hash = get_slice(id, 0);
+    for (size_t i = 1; i < slices; ++i) {
+      combine(hash, get_slice(id, i));
+    }
+    return hash;

Review Comment:
   we could use `gsl::span` here, it would also verify convertibility
   ```suggestion
       for (size_t slice : gsl::span(id.data_).as_span<const size_t>()) {
         combine(hash, slice);
       }
   ```



-- 
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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r905313631


##########
METRICS.md:
##########
@@ -0,0 +1,155 @@
+<!--
+  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.
+-->
+
+# Apache NiFi - MiNiFi - C++ Metrics Readme.
+
+
+This readme defines the metrics published by Apache NiFi. All options defined are located in minifi.properties.
+
+## Table of Contents
+
+- [Description](#description)
+- [Configuration](#configuration)
+- [Metrics](#metrics)
+
+## Description
+
+Apache NiFi MiNiFi C++ can communicate metrics about the agent's status, that can be a system level or component level metric.
+These metrics are exposed through the agent implemented metric publishers that can be configured in the minifi.properties.
+Aside from the publisher exposed metrics, metrics are also sent through C2 protocol of which there is more information in the
+[C2 documentation](C2.md#metrics).
+
+## Configuration
+
+To configure the a metrics publisher first we have to set which publisher class should be used:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.class=PrometheusMetricsPublisher
+
+Currently PrometheusMetricsPublisher is the only available publisher in MiNiFi C++ which publishes metrics to a Prometheus server.
+To use the publisher a port should also be configured where the metrics will be available to be scraped through:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.PrometheusMetricsPublisher.port=9936
+
+The last option defines which metric classes should be exposed through the metrics publisher in configured with a comma separated value:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.metrics=QueueMetrics,RepositoryMetrics,GetFileMetrics,DeviceInfoNode,FlowInformation
+
+## Metrics
+
+The following section defines the currently available metrics to be published by the MiNiFi C++ agent.
+
+NOTE: In Prometheus all metrics are extended with a `minifi_` prefix to mark the domain of the metric. For example the `connection_name` metric is published as `minifi_connection_name` in Prometheus.
+
+### QueueMetrics
+
+QueueMetrics is a system level metric that reports queue metrics for every connection in the flow.
+
+| Metric name          | Labels                                         | Description                                |
+|----------------------|------------------------------------------------|--------------------------------------------|
+| queue_data_size      | metric_class, connection_uuid, connection_name | Max queue size to apply back pressure      |
+| queue_data_size_max  | metric_class, connection_uuid, connection_name | Max queue data size to apply back pressure |
+| queue_size           | metric_class, connection_uuid, connection_name | Current queue size                         |
+| queue_size_max       | metric_class, connection_uuid, connection_name | Current queue data size                    |

Review Comment:
   I realize it would be hard to have metric sink-specific names without some type metadata for metrics, and I wouldn't want to touch the C2 protocol. Let's leave it as is for now, and think about changing the prometheus metrics names later, after some metrics-related refactoring.
   
   Could you check the descriptions though? They are mixed up.



-- 
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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r906188273


##########
libminifi/include/FlowController.h:
##########
@@ -252,8 +255,7 @@ class FlowController : public core::controller::ForwardingControllerServiceProvi
   // Thread pool for schedulers
   utils::ThreadPool<utils::TaskRescheduleInfo> thread_pool_;
   std::map<utils::Identifier, std::unique_ptr<state::ProcessorController>> processor_to_controller_;
+  std::unique_ptr<state::MetricsPublisher> metrics_publisher_;

Review Comment:
   That's fine, I've created the followup jria: https://issues.apache.org/jira/browse/MINIFICPP-1869



-- 
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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r905858178


##########
libminifi/include/core/state/nodes/RepositoryMetrics.h:
##########
@@ -90,15 +87,18 @@ class RepositoryMetrics : public ResponseNode {
     return serialized;
   }
 
+  std::vector<PublishedMetric> calculateMetrics() override {
+    std::vector<PublishedMetric> metrics;
+    for (const auto& [_, repo] : repositories_) {
+      metrics.push_back({"is_running", (repo->isRunning() ? 1.0 : 0.0), {{"metric_class", getName()}, {"repository_name", repo->getName()}}});
+      metrics.push_back({"is_full", (repo->isFull() ? 1.0 : 0.0), {{"metric_class", getName()}, {"repository_name", repo->getName()}}});
+      metrics.push_back({"repository_size", static_cast<double>(repo->getRepoSize()), {{"metric_class", getName()}, {"repository_name", repo->getName()}}});
+    }
+    return metrics;
+  }

Review Comment:
   Thanks, now I see what you mean. It would be better if they would be interchangeable somehow, but I think it would be problematic to implement it at the moment. Currently the common denominator between publishing through metrics exposers and through C2 is the `ResponseNode` which is passed to both to Prometheus and the C2Client, but they work quite differently through the `serialize` and the `calculateMetrics` as they do more now than just formatting.
   
   The `serialize` member defines a tree structure of the metrics, how they should be aligned which returns a `SerializedResponseNode` (which is then moved to `C2Payload` objects, then those are transformed to json objects by rapidjsons so it's quite complex, something we may need to rewrite in the future). Opposed to this the calculateMetrics returns a flat format with key-value pairs with additional labels, so it's structured quite differently. We also have an `ObjectNode` responsenode for mimicing the C2 properties' tree structure containing children nodes. Its serialization process goes through the nodes recursively which requires to have this `serialize` as a member function so it would be hard to move it to the C2Client. 
   
   Also moving the serialization and the metrics calculation to C2Client and the metrics publisher would require us to define parts of a newly introduced metric in 3 places, which would not be really intuitive.
   
   Another problem is that both members contain different metrics as `calculateMetrics` only collects quantifiable data, that can be represented as a double for monitoring tools, while `serialize` contains string data as well for example.



##########
METRICS.md:
##########
@@ -0,0 +1,155 @@
+<!--
+  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.
+-->
+
+# Apache NiFi - MiNiFi - C++ Metrics Readme.
+
+
+This readme defines the metrics published by Apache NiFi. All options defined are located in minifi.properties.
+
+## Table of Contents
+
+- [Description](#description)
+- [Configuration](#configuration)
+- [Metrics](#metrics)
+
+## Description
+
+Apache NiFi MiNiFi C++ can communicate metrics about the agent's status, that can be a system level or component level metric.
+These metrics are exposed through the agent implemented metric publishers that can be configured in the minifi.properties.
+Aside from the publisher exposed metrics, metrics are also sent through C2 protocol of which there is more information in the
+[C2 documentation](C2.md#metrics).
+
+## Configuration
+
+To configure the a metrics publisher first we have to set which publisher class should be used:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.class=PrometheusMetricsPublisher
+
+Currently PrometheusMetricsPublisher is the only available publisher in MiNiFi C++ which publishes metrics to a Prometheus server.
+To use the publisher a port should also be configured where the metrics will be available to be scraped through:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.PrometheusMetricsPublisher.port=9936
+
+The last option defines which metric classes should be exposed through the metrics publisher in configured with a comma separated value:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.metrics=QueueMetrics,RepositoryMetrics,GetFileMetrics,DeviceInfoNode,FlowInformation
+
+## Metrics
+
+The following section defines the currently available metrics to be published by the MiNiFi C++ agent.
+
+NOTE: In Prometheus all metrics are extended with a `minifi_` prefix to mark the domain of the metric. For example the `connection_name` metric is published as `minifi_connection_name` in Prometheus.
+
+### QueueMetrics
+
+QueueMetrics is a system level metric that reports queue metrics for every connection in the flow.
+
+| Metric name          | Labels                                         | Description                                |
+|----------------------|------------------------------------------------|--------------------------------------------|
+| queue_data_size      | metric_class, connection_uuid, connection_name | Max queue size to apply back pressure      |
+| queue_data_size_max  | metric_class, connection_uuid, connection_name | Max queue data size to apply back pressure |
+| queue_size           | metric_class, connection_uuid, connection_name | Current queue size                         |
+| queue_size_max       | metric_class, connection_uuid, connection_name | Current queue data size                    |

Review Comment:
   Okay, I think we would have more input and would be able to generalize these things a bit easier if we had another monitoring tool support, that would make it easier to see what needs to be more generalized.
   
   I see there is a mixup in the descriptions I'll fix 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.

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] adamdebreceni commented on a diff in pull request #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r907497990


##########
libminifi/include/utils/Id.h:
##########
@@ -132,3 +134,26 @@ class NonRepeatingStringGenerator {
 };
 
 }  // namespace org::apache::nifi::minifi::utils
+
+namespace std {
+template<>
+struct hash<org::apache::nifi::minifi::utils::Identifier> {
+  size_t operator()(const org::apache::nifi::minifi::utils::Identifier& id) const noexcept {
+    constexpr int slices = sizeof(org::apache::nifi::minifi::utils::Identifier) / sizeof(size_t);
+    const auto combine = [](size_t& seed, size_t new_hash) {
+      // from the boost hash_combine docs
+      seed ^= new_hash + 0x9e3779b9 + (seed << 6) + (seed >> 2);
+    };
+    const auto get_slice = [](const org::apache::nifi::minifi::utils::Identifier& id, size_t idx) -> size_t {
+      size_t result{};
+      memcpy(&result, reinterpret_cast<const unsigned char*>(&id.data_) + idx * sizeof(size_t), sizeof(size_t));
+      return result;
+    };
+    size_t hash = get_slice(id, 0);
+    for (size_t i = 1; i < slices; ++i) {
+      combine(hash, get_slice(id, i));
+    }
+    return hash;

Review Comment:
   ~we could use `gsl::span` here, it would also verify convertibility~ (Edit: no we couldn't)



-- 
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] adamdebreceni closed pull request #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
adamdebreceni closed pull request #1340: MINIFICPP-1829 Export metrics for use with Prometheus
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340


-- 
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] adamdebreceni commented on a diff in pull request #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r908092950


##########
libminifi/src/FlowController.cpp:
##########
@@ -150,6 +154,7 @@ bool FlowController::applyConfiguration(const std::string &source, const std::st
     this->root_ = std::move(prevRoot);

Review Comment:
   won't unlocking at line 140 make it possible for this line to execute without a valid lock (due to an exception after the unlock)?



-- 
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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r908167577


##########
libminifi/src/FlowController.cpp:
##########
@@ -150,6 +154,7 @@ bool FlowController::applyConfiguration(const std::string &source, const std::st
     this->root_ = std::move(prevRoot);

Review Comment:
   Good idea, updated in b53169315c1ec8d6222e8e55efab8631c75325c5



-- 
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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r892504581


##########
libminifi/include/core/state/nodes/QueueMetrics.h:
##########
@@ -94,15 +82,20 @@ class QueueMetrics : public ResponseNode {
     return serialized;
   }
 
- protected:
-  std::map<std::string, std::unique_ptr<minifi::Connection>> connections;
+  std::vector<PublishedMetric> calculateMetrics() override {
+    std::vector<PublishedMetric> metrics;
+    for (const auto& [_, connection] : connections_) {
+      metrics.push_back({"queue_data_size", static_cast<double>(connection->getQueueDataSize()),
+        {{"connection_uuid", connection->getUUIDStr()}, {"connection_name", connection->getName()}, {"metric_class", getName()}}});
+      metrics.push_back({"queue_data_size_max", static_cast<double>(connection->getMaxQueueDataSize()),
+        {{"connection_uuid", connection->getUUIDStr()}, {"connection_name", connection->getName()}, {"metric_class", getName()}}});
+      metrics.push_back({"queue_size", static_cast<double>(connection->getQueueSize()),
+        {{"connection_uuid", connection->getUUIDStr()}, {"connection_name", connection->getName()}, {"metric_class", getName()}}});
+      metrics.push_back({"queue_size_max", static_cast<double>(connection->getMaxQueueSize()),
+        {{"connection_uuid", connection->getUUIDStr()}, {"connection_name", connection->getName()}, {"metric_class", getName()}}});
+    }
+    return metrics;

Review Comment:
   It would be nice if the Prometheus publisher could take the structured data from FlowInformation and flatten it to the Prometheus-friendly format internally.



-- 
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] adamdebreceni commented on a diff in pull request #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r892443379


##########
extensions/prometheus/PrometheusExposerWrapper.cpp:
##########
@@ -0,0 +1,34 @@
+/**
+ * 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 "PrometheusExposerWrapper.h"
+
+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 %u", port);

Review Comment:
   `PRIu32` might be the appropriate format 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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r905120725


##########
libminifi/include/core/state/nodes/RepositoryMetrics.h:
##########
@@ -90,15 +87,18 @@ class RepositoryMetrics : public ResponseNode {
     return serialized;
   }
 
+  std::vector<PublishedMetric> calculateMetrics() override {
+    std::vector<PublishedMetric> metrics;
+    for (const auto& [_, repo] : repositories_) {
+      metrics.push_back({"is_running", (repo->isRunning() ? 1.0 : 0.0), {{"metric_class", getName()}, {"repository_name", repo->getName()}}});
+      metrics.push_back({"is_full", (repo->isFull() ? 1.0 : 0.0), {{"metric_class", getName()}, {"repository_name", repo->getName()}}});
+      metrics.push_back({"repository_size", static_cast<double>(repo->getRepoSize()), {{"metric_class", getName()}, {"repository_name", repo->getName()}}});
+    }
+    return metrics;
+  }

Review Comment:
   Could you elaborate a bit more please? What do you think is currently too Prometheus specific format here that should be changed and transformed later? Also what do you mean by unifying `serialize`? The serialized nodes are not used by Prometheus only on the C2 protocol.



##########
libminifi/include/properties/Configuration.h:
##########
@@ -156,12 +156,17 @@ class Configuration : public Properties {
 
   static constexpr const char *nifi_asset_directory = "nifi.asset.directory";
 
+  // Metrics publisher options
+  static constexpr const char *nifi_metrics_publisher_class = "nifi.metrics.publisher.class";
+  static constexpr const char *nifi_metrics_publisher_prometheus_metrics_publisher_port = "nifi.metrics.publisher.PrometheusMetricsPublisher.port";

Review Comment:
   The problem is that all possible configuration options sent through C2 are retrieved from here, so if we want to advertise the prometheus options as well, they should be added here.



##########
METRICS.md:
##########
@@ -0,0 +1,155 @@
+<!--
+  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.
+-->
+
+# Apache NiFi - MiNiFi - C++ Metrics Readme.
+
+
+This readme defines the metrics published by Apache NiFi. All options defined are located in minifi.properties.
+
+## Table of Contents
+
+- [Description](#description)
+- [Configuration](#configuration)
+- [Metrics](#metrics)
+
+## Description
+
+Apache NiFi MiNiFi C++ can communicate metrics about the agent's status, that can be a system level or component level metric.
+These metrics are exposed through the agent implemented metric publishers that can be configured in the minifi.properties.
+Aside from the publisher exposed metrics, metrics are also sent through C2 protocol of which there is more information in the
+[C2 documentation](C2.md#metrics).
+
+## Configuration
+
+To configure the a metrics publisher first we have to set which publisher class should be used:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.class=PrometheusMetricsPublisher
+
+Currently PrometheusMetricsPublisher is the only available publisher in MiNiFi C++ which publishes metrics to a Prometheus server.
+To use the publisher a port should also be configured where the metrics will be available to be scraped through:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.PrometheusMetricsPublisher.port=9936
+
+The last option defines which metric classes should be exposed through the metrics publisher in configured with a comma separated value:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.metrics=QueueMetrics,RepositoryMetrics,GetFileMetrics,DeviceInfoNode,FlowInformation
+
+## Metrics
+
+The following section defines the currently available metrics to be published by the MiNiFi C++ agent.
+
+NOTE: In Prometheus all metrics are extended with a `minifi_` prefix to mark the domain of the metric. For example the `connection_name` metric is published as `minifi_connection_name` in Prometheus.
+
+### QueueMetrics
+
+QueueMetrics is a system level metric that reports queue metrics for every connection in the flow.
+
+| Metric name          | Labels                                         | Description                                |
+|----------------------|------------------------------------------------|--------------------------------------------|
+| queue_data_size      | metric_class, connection_uuid, connection_name | Max queue size to apply back pressure      |
+| queue_data_size_max  | metric_class, connection_uuid, connection_name | Max queue data size to apply back pressure |
+| queue_size           | metric_class, connection_uuid, connection_name | Current queue size                         |
+| queue_size_max       | metric_class, connection_uuid, connection_name | Current queue data size                    |

Review Comment:
   What would you suggest chaning in the names? The 'minifi' prefix is applied in the Prometheus extension, I intended these names to be used in other metric collectors as well, so I only added a note above that in Prometheus these are extended with the `minifi` prefix. Do you think we should add a suffix here for the units?



-- 
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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r905895781


##########
libminifi/src/c2/C2Client.cpp:
##########
@@ -206,35 +130,13 @@ void C2Client::loadC2ResponseConfiguration(const std::string &prefix) {
       }
       std::shared_ptr<state::response::ResponseNode> new_node = std::make_shared<state::response::ObjectNode>(name);
       if (configuration_->get(classOption, class_definitions)) {
-        std::vector<std::string> classes = utils::StringUtils::split(class_definitions, ",");
-        for (const std::string& clazz : classes) {
-          // instantiate the object
-          std::shared_ptr<core::CoreComponent> ptr = core::ClassLoader::getDefaultClassLoader().instantiate(clazz, clazz);
-          if (nullptr == ptr) {
-            const bool found_metric = [&] {
-              std::lock_guard<std::mutex> guard{metrics_mutex_};
-              auto metric = component_metrics_.find(clazz);
-              if (metric != component_metrics_.end()) {
-                ptr = metric->second;
-                return true;
-              }
-              return false;
-            }();
-            if (!found_metric) {
-              logger_->log_error("No metric defined for %s", clazz);
-              continue;
-            }
-          }
-          auto node = std::dynamic_pointer_cast<state::response::ResponseNode>(ptr);
-          std::static_pointer_cast<state::response::ObjectNode>(new_node)->add_node(node);
-        }
-
+        loadNodeClasses(class_definitions, new_node);
       } else {
         std::string optionName = option + "." + name;
-        auto node = loadC2ResponseConfiguration(optionName, new_node);
+        loadC2ResponseConfiguration(optionName, new_node);
       }
 
-      std::lock_guard<std::mutex> guard{metrics_mutex_};
+      // We don't need to lock here we do it in the initializeResponseNodes

Review Comment:
   Updated in afe6e3ec99c2c21454fd840d031677a2b948f1bd



##########
libminifi/include/core/state/ConnectionStore.h:
##########
@@ -0,0 +1,60 @@
+/**
+ *
+ * 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 <map>
+#include <string>
+#include <vector>
+
+#include "Connection.h"
+#include "utils/gsl.h"
+
+namespace org::apache::nifi::minifi::state {
+
+class ConnectionStore {
+ public:
+  void updateConnection(minifi::Connection* connection) {
+    if (nullptr != connection) {
+      connections_[connection->getUUIDStr()] = connection;
+    }
+  }
+
+  std::vector<PublishedMetric> calculateConnectionMetrics(const std::string& metric_class) {
+    std::vector<PublishedMetric> metrics;
+
+    for (const auto& [_, connection] : connections_) {
+      metrics.push_back({"queue_data_size", static_cast<double>(connection->getQueueDataSize()),
+        {{"connection_uuid", connection->getUUIDStr()}, {"connection_name", connection->getName()}, {"metric_class", metric_class}}});
+      metrics.push_back({"queue_data_size_max", static_cast<double>(connection->getMaxQueueDataSize()),
+        {{"connection_uuid", connection->getUUIDStr()}, {"connection_name", connection->getName()}, {"metric_class", metric_class}}});
+      metrics.push_back({"queue_size", static_cast<double>(connection->getQueueSize()),
+        {{"connection_uuid", connection->getUUIDStr()}, {"connection_name", connection->getName()}, {"metric_class", metric_class}}});
+      metrics.push_back({"queue_size_max", static_cast<double>(connection->getMaxQueueSize()),
+        {{"connection_uuid", connection->getUUIDStr()}, {"connection_name", connection->getName()}, {"metric_class", metric_class}}});
+    }
+
+    return metrics;
+  }
+
+  virtual ~ConnectionStore() = default;
+
+ protected:
+  std::map<std::string, minifi::Connection*> connections_;

Review Comment:
   Thanks! Updated in afe6e3ec99c2c21454fd840d031677a2b948f1bd



##########
METRICS.md:
##########
@@ -0,0 +1,155 @@
+<!--
+  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.
+-->
+
+# Apache NiFi - MiNiFi - C++ Metrics Readme.
+
+
+This readme defines the metrics published by Apache NiFi. All options defined are located in minifi.properties.
+
+## Table of Contents
+
+- [Description](#description)
+- [Configuration](#configuration)
+- [Metrics](#metrics)
+
+## Description
+
+Apache NiFi MiNiFi C++ can communicate metrics about the agent's status, that can be a system level or component level metric.
+These metrics are exposed through the agent implemented metric publishers that can be configured in the minifi.properties.
+Aside from the publisher exposed metrics, metrics are also sent through C2 protocol of which there is more information in the
+[C2 documentation](C2.md#metrics).
+
+## Configuration
+
+To configure the a metrics publisher first we have to set which publisher class should be used:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.class=PrometheusMetricsPublisher
+
+Currently PrometheusMetricsPublisher is the only available publisher in MiNiFi C++ which publishes metrics to a Prometheus server.
+To use the publisher a port should also be configured where the metrics will be available to be scraped through:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.PrometheusMetricsPublisher.port=9936
+
+The last option defines which metric classes should be exposed through the metrics publisher in configured with a comma separated value:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.metrics=QueueMetrics,RepositoryMetrics,GetFileMetrics,DeviceInfoNode,FlowInformation
+
+## Metrics
+
+The following section defines the currently available metrics to be published by the MiNiFi C++ agent.
+
+NOTE: In Prometheus all metrics are extended with a `minifi_` prefix to mark the domain of the metric. For example the `connection_name` metric is published as `minifi_connection_name` in Prometheus.
+
+### QueueMetrics
+
+QueueMetrics is a system level metric that reports queue metrics for every connection in the flow.
+
+| Metric name          | Labels                                         | Description                                |
+|----------------------|------------------------------------------------|--------------------------------------------|
+| queue_data_size      | metric_class, connection_uuid, connection_name | Max queue size to apply back pressure      |
+| queue_data_size_max  | metric_class, connection_uuid, connection_name | Max queue data size to apply back pressure |
+| queue_size           | metric_class, connection_uuid, connection_name | Current queue size                         |
+| queue_size_max       | metric_class, connection_uuid, connection_name | Current queue data size                    |

Review Comment:
   Updated in afe6e3ec99c2c21454fd840d031677a2b948f1bd



-- 
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] adamdebreceni commented on a diff in pull request #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r907497990


##########
libminifi/include/utils/Id.h:
##########
@@ -132,3 +134,26 @@ class NonRepeatingStringGenerator {
 };
 
 }  // namespace org::apache::nifi::minifi::utils
+
+namespace std {
+template<>
+struct hash<org::apache::nifi::minifi::utils::Identifier> {
+  size_t operator()(const org::apache::nifi::minifi::utils::Identifier& id) const noexcept {
+    constexpr int slices = sizeof(org::apache::nifi::minifi::utils::Identifier) / sizeof(size_t);
+    const auto combine = [](size_t& seed, size_t new_hash) {
+      // from the boost hash_combine docs
+      seed ^= new_hash + 0x9e3779b9 + (seed << 6) + (seed >> 2);
+    };
+    const auto get_slice = [](const org::apache::nifi::minifi::utils::Identifier& id, size_t idx) -> size_t {
+      size_t result{};
+      memcpy(&result, reinterpret_cast<const unsigned char*>(&id.data_) + idx * sizeof(size_t), sizeof(size_t));
+      return result;
+    };
+    size_t hash = get_slice(id, 0);
+    for (size_t i = 1; i < slices; ++i) {
+      combine(hash, get_slice(id, i));
+    }
+    return hash;

Review Comment:
   ~we could use `gsl::span` here, it would also verify convertibility~ (Edit: no we couldn't)
   ```
       for (size_t slice : gsl::span(id.data_).as_span<const size_t>()) {
         combine(hash, slice);
       }
   ```



-- 
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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r905858242


##########
METRICS.md:
##########
@@ -0,0 +1,155 @@
+<!--
+  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.
+-->
+
+# Apache NiFi - MiNiFi - C++ Metrics Readme.
+
+
+This readme defines the metrics published by Apache NiFi. All options defined are located in minifi.properties.
+
+## Table of Contents
+
+- [Description](#description)
+- [Configuration](#configuration)
+- [Metrics](#metrics)
+
+## Description
+
+Apache NiFi MiNiFi C++ can communicate metrics about the agent's status, that can be a system level or component level metric.
+These metrics are exposed through the agent implemented metric publishers that can be configured in the minifi.properties.
+Aside from the publisher exposed metrics, metrics are also sent through C2 protocol of which there is more information in the
+[C2 documentation](C2.md#metrics).
+
+## Configuration
+
+To configure the a metrics publisher first we have to set which publisher class should be used:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.class=PrometheusMetricsPublisher
+
+Currently PrometheusMetricsPublisher is the only available publisher in MiNiFi C++ which publishes metrics to a Prometheus server.
+To use the publisher a port should also be configured where the metrics will be available to be scraped through:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.PrometheusMetricsPublisher.port=9936
+
+The last option defines which metric classes should be exposed through the metrics publisher in configured with a comma separated value:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.metrics=QueueMetrics,RepositoryMetrics,GetFileMetrics,DeviceInfoNode,FlowInformation
+
+## Metrics
+
+The following section defines the currently available metrics to be published by the MiNiFi C++ agent.
+
+NOTE: In Prometheus all metrics are extended with a `minifi_` prefix to mark the domain of the metric. For example the `connection_name` metric is published as `minifi_connection_name` in Prometheus.
+
+### QueueMetrics
+
+QueueMetrics is a system level metric that reports queue metrics for every connection in the flow.
+
+| Metric name          | Labels                                         | Description                                |
+|----------------------|------------------------------------------------|--------------------------------------------|
+| queue_data_size      | metric_class, connection_uuid, connection_name | Max queue size to apply back pressure      |
+| queue_data_size_max  | metric_class, connection_uuid, connection_name | Max queue data size to apply back pressure |
+| queue_size           | metric_class, connection_uuid, connection_name | Current queue size                         |
+| queue_size_max       | metric_class, connection_uuid, connection_name | Current queue data size                    |

Review Comment:
   Okay, I think we would have more input and would be able to generalize these things a bit easier if we had another monitoring tool support, that would make it easier to see what needs to be more generalized.
   
   I see there is a mixup in the descriptions, thanks, I'll fix 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.

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] adamdebreceni commented on a diff in pull request #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r908132236


##########
libminifi/src/FlowController.cpp:
##########
@@ -150,6 +154,7 @@ bool FlowController::applyConfiguration(const std::string &source, const std::st
     this->root_ = std::move(prevRoot);

Review Comment:
   what was the problem that required manual locking/unlocking?



-- 
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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r905858242


##########
METRICS.md:
##########
@@ -0,0 +1,155 @@
+<!--
+  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.
+-->
+
+# Apache NiFi - MiNiFi - C++ Metrics Readme.
+
+
+This readme defines the metrics published by Apache NiFi. All options defined are located in minifi.properties.
+
+## Table of Contents
+
+- [Description](#description)
+- [Configuration](#configuration)
+- [Metrics](#metrics)
+
+## Description
+
+Apache NiFi MiNiFi C++ can communicate metrics about the agent's status, that can be a system level or component level metric.
+These metrics are exposed through the agent implemented metric publishers that can be configured in the minifi.properties.
+Aside from the publisher exposed metrics, metrics are also sent through C2 protocol of which there is more information in the
+[C2 documentation](C2.md#metrics).
+
+## Configuration
+
+To configure the a metrics publisher first we have to set which publisher class should be used:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.class=PrometheusMetricsPublisher
+
+Currently PrometheusMetricsPublisher is the only available publisher in MiNiFi C++ which publishes metrics to a Prometheus server.
+To use the publisher a port should also be configured where the metrics will be available to be scraped through:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.PrometheusMetricsPublisher.port=9936
+
+The last option defines which metric classes should be exposed through the metrics publisher in configured with a comma separated value:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.metrics=QueueMetrics,RepositoryMetrics,GetFileMetrics,DeviceInfoNode,FlowInformation
+
+## Metrics
+
+The following section defines the currently available metrics to be published by the MiNiFi C++ agent.
+
+NOTE: In Prometheus all metrics are extended with a `minifi_` prefix to mark the domain of the metric. For example the `connection_name` metric is published as `minifi_connection_name` in Prometheus.
+
+### QueueMetrics
+
+QueueMetrics is a system level metric that reports queue metrics for every connection in the flow.
+
+| Metric name          | Labels                                         | Description                                |
+|----------------------|------------------------------------------------|--------------------------------------------|
+| queue_data_size      | metric_class, connection_uuid, connection_name | Max queue size to apply back pressure      |
+| queue_data_size_max  | metric_class, connection_uuid, connection_name | Max queue data size to apply back pressure |
+| queue_size           | metric_class, connection_uuid, connection_name | Current queue size                         |
+| queue_size_max       | metric_class, connection_uuid, connection_name | Current queue data size                    |

Review Comment:
   Okay, I think we would have more input and would be able to generalize these things a bit easier if we had another monitoring tool support. Maybe we can revise more of this then.
   
   I see there is a mixup in the descriptions, thanks, I'll fix 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.

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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r905895794


##########
libminifi/include/core/state/ConnectionStore.h:
##########
@@ -0,0 +1,60 @@
+/**
+ *
+ * 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 <map>
+#include <string>
+#include <vector>
+
+#include "Connection.h"
+#include "utils/gsl.h"
+
+namespace org::apache::nifi::minifi::state {
+
+class ConnectionStore {
+ public:
+  void updateConnection(minifi::Connection* connection) {
+    if (nullptr != connection) {
+      connections_[connection->getUUIDStr()] = connection;
+    }
+  }
+
+  std::vector<PublishedMetric> calculateConnectionMetrics(const std::string& metric_class) {
+    std::vector<PublishedMetric> metrics;
+
+    for (const auto& [_, connection] : connections_) {
+      metrics.push_back({"queue_data_size", static_cast<double>(connection->getQueueDataSize()),
+        {{"connection_uuid", connection->getUUIDStr()}, {"connection_name", connection->getName()}, {"metric_class", metric_class}}});
+      metrics.push_back({"queue_data_size_max", static_cast<double>(connection->getMaxQueueDataSize()),
+        {{"connection_uuid", connection->getUUIDStr()}, {"connection_name", connection->getName()}, {"metric_class", metric_class}}});
+      metrics.push_back({"queue_size", static_cast<double>(connection->getQueueSize()),
+        {{"connection_uuid", connection->getUUIDStr()}, {"connection_name", connection->getName()}, {"metric_class", metric_class}}});
+      metrics.push_back({"queue_size_max", static_cast<double>(connection->getMaxQueueSize()),
+        {{"connection_uuid", connection->getUUIDStr()}, {"connection_name", connection->getName()}, {"metric_class", metric_class}}});
+    }
+
+    return metrics;
+  }
+
+  virtual ~ConnectionStore() = default;
+
+ protected:
+  std::map<std::string, minifi::Connection*> connections_;

Review Comment:
   Thanks! Updated in af29dc57c0681f8c632a1f057ce16b5b4737f4a6



##########
libminifi/src/c2/C2Client.cpp:
##########
@@ -206,35 +130,13 @@ void C2Client::loadC2ResponseConfiguration(const std::string &prefix) {
       }
       std::shared_ptr<state::response::ResponseNode> new_node = std::make_shared<state::response::ObjectNode>(name);
       if (configuration_->get(classOption, class_definitions)) {
-        std::vector<std::string> classes = utils::StringUtils::split(class_definitions, ",");
-        for (const std::string& clazz : classes) {
-          // instantiate the object
-          std::shared_ptr<core::CoreComponent> ptr = core::ClassLoader::getDefaultClassLoader().instantiate(clazz, clazz);
-          if (nullptr == ptr) {
-            const bool found_metric = [&] {
-              std::lock_guard<std::mutex> guard{metrics_mutex_};
-              auto metric = component_metrics_.find(clazz);
-              if (metric != component_metrics_.end()) {
-                ptr = metric->second;
-                return true;
-              }
-              return false;
-            }();
-            if (!found_metric) {
-              logger_->log_error("No metric defined for %s", clazz);
-              continue;
-            }
-          }
-          auto node = std::dynamic_pointer_cast<state::response::ResponseNode>(ptr);
-          std::static_pointer_cast<state::response::ObjectNode>(new_node)->add_node(node);
-        }
-
+        loadNodeClasses(class_definitions, new_node);
       } else {
         std::string optionName = option + "." + name;
-        auto node = loadC2ResponseConfiguration(optionName, new_node);
+        loadC2ResponseConfiguration(optionName, new_node);
       }
 
-      std::lock_guard<std::mutex> guard{metrics_mutex_};
+      // We don't need to lock here we do it in the initializeResponseNodes

Review Comment:
   Updated in af29dc57c0681f8c632a1f057ce16b5b4737f4a6



##########
METRICS.md:
##########
@@ -0,0 +1,155 @@
+<!--
+  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.
+-->
+
+# Apache NiFi - MiNiFi - C++ Metrics Readme.
+
+
+This readme defines the metrics published by Apache NiFi. All options defined are located in minifi.properties.
+
+## Table of Contents
+
+- [Description](#description)
+- [Configuration](#configuration)
+- [Metrics](#metrics)
+
+## Description
+
+Apache NiFi MiNiFi C++ can communicate metrics about the agent's status, that can be a system level or component level metric.
+These metrics are exposed through the agent implemented metric publishers that can be configured in the minifi.properties.
+Aside from the publisher exposed metrics, metrics are also sent through C2 protocol of which there is more information in the
+[C2 documentation](C2.md#metrics).
+
+## Configuration
+
+To configure the a metrics publisher first we have to set which publisher class should be used:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.class=PrometheusMetricsPublisher
+
+Currently PrometheusMetricsPublisher is the only available publisher in MiNiFi C++ which publishes metrics to a Prometheus server.
+To use the publisher a port should also be configured where the metrics will be available to be scraped through:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.PrometheusMetricsPublisher.port=9936
+
+The last option defines which metric classes should be exposed through the metrics publisher in configured with a comma separated value:
+
+	# in minifi.properties
+
+	nifi.metrics.publisher.metrics=QueueMetrics,RepositoryMetrics,GetFileMetrics,DeviceInfoNode,FlowInformation
+
+## Metrics
+
+The following section defines the currently available metrics to be published by the MiNiFi C++ agent.
+
+NOTE: In Prometheus all metrics are extended with a `minifi_` prefix to mark the domain of the metric. For example the `connection_name` metric is published as `minifi_connection_name` in Prometheus.
+
+### QueueMetrics
+
+QueueMetrics is a system level metric that reports queue metrics for every connection in the flow.
+
+| Metric name          | Labels                                         | Description                                |
+|----------------------|------------------------------------------------|--------------------------------------------|
+| queue_data_size      | metric_class, connection_uuid, connection_name | Max queue size to apply back pressure      |
+| queue_data_size_max  | metric_class, connection_uuid, connection_name | Max queue data size to apply back pressure |
+| queue_size           | metric_class, connection_uuid, connection_name | Current queue size                         |
+| queue_size_max       | metric_class, connection_uuid, connection_name | Current queue data size                    |

Review Comment:
   Updated in af29dc57c0681f8c632a1f057ce16b5b4737f4a6



-- 
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 #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r888784621


##########
libminifi/src/core/state/nodes/QueueMetrics.cpp:
##########
@@ -0,0 +1,27 @@
+/**
+ *
+ * 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 "core/state/nodes/QueueMetrics.h"
+#include "core/Resource.h"
+
+namespace org::apache::nifi::minifi::state::response {
+
+REGISTER_RESOURCE(QueueMetrics, "Node part of an AST that defines queue metric information");

Review Comment:
   What does this mean? "Node part of an AST"
   I know the meaning of each word, but not the whole doesn't make sense to me.



##########
docker/test/integration/minifi/core/DockerTestCluster.py:
##########
@@ -283,3 +282,79 @@ def write_content_to_container(self, content, dst):
                 tar.addfile(info, io.BytesIO(content.encode('utf-8')))
             with open(os.path.join(td, 'content.tar'), 'rb') as data:
                 return container.put_archive(os.path.dirname(dst_path), data.read())
+
+    def check_metric_class_on_prometheus(self, metric_class, timeout_seconds):
+        start_time = time.perf_counter()
+        while (time.perf_counter() - start_time) < timeout_seconds:
+            if self.verify_metric_class(metric_class):
+                return True
+            time.sleep(1)
+        return False

Review Comment:
   I suggest adding the word "wait" somewhere in functions that wait for a condition to occur with a timeout. Maybe "wait_for_metric_class_on_prometheus"?



##########
libminifi/include/FlowController.h:
##########
@@ -252,8 +255,7 @@ class FlowController : public core::controller::ForwardingControllerServiceProvi
   // Thread pool for schedulers
   utils::ThreadPool<utils::TaskRescheduleInfo> thread_pool_;
   std::map<utils::Identifier, std::unique_ptr<state::ProcessorController>> processor_to_controller_;
+  std::unique_ptr<state::MetricsPublisher> metrics_publisher_;

Review Comment:
   The flow controller shouldn't own this service, or even depend on it. Each entity that's capable of exporting metrics should expose that on its public interface. The flow controller may provide some aggregation into a single structure if necessary, but it shouldn't do much else.
   Let's have the MetricsPublisher depend on a metric source callback, that's bound to this aggregation member function of the flow controller, without each depending on the other.



-- 
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] adamdebreceni commented on a diff in pull request #1340: MINIFICPP-1829 Export metrics for use with Prometheus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1340:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1340#discussion_r892486920


##########
libminifi/include/core/state/nodes/QueueMetrics.h:
##########
@@ -94,15 +82,20 @@ class QueueMetrics : public ResponseNode {
     return serialized;
   }
 
- protected:
-  std::map<std::string, std::unique_ptr<minifi::Connection>> connections;
+  std::vector<PublishedMetric> calculateMetrics() override {
+    std::vector<PublishedMetric> metrics;
+    for (const auto& [_, connection] : connections_) {
+      metrics.push_back({"queue_data_size", static_cast<double>(connection->getQueueDataSize()),
+        {{"connection_uuid", connection->getUUIDStr()}, {"connection_name", connection->getName()}, {"metric_class", getName()}}});
+      metrics.push_back({"queue_data_size_max", static_cast<double>(connection->getMaxQueueDataSize()),
+        {{"connection_uuid", connection->getUUIDStr()}, {"connection_name", connection->getName()}, {"metric_class", getName()}}});
+      metrics.push_back({"queue_size", static_cast<double>(connection->getQueueSize()),
+        {{"connection_uuid", connection->getUUIDStr()}, {"connection_name", connection->getName()}, {"metric_class", getName()}}});
+      metrics.push_back({"queue_size_max", static_cast<double>(connection->getMaxQueueSize()),
+        {{"connection_uuid", connection->getUUIDStr()}, {"connection_name", connection->getName()}, {"metric_class", getName()}}});
+    }
+    return metrics;

Review Comment:
   this seems to be similar to the one in `FlowInformation` could we somehow unify this part?



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