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

[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1532: MINIFICPP-2076 Implement logging metrics publisher

fgerlits commented on code in PR #1532:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1532#discussion_r1213108449


##########
METRICS.md:
##########
@@ -42,25 +42,66 @@ Aside from the publisher exposed metrics, metrics are also sent through C2 proto
 
 ## Configuration
 
-To configure the a metrics publisher first we have to set which publisher class should be used:
+Currently LogMetricsPublisher and PrometheusMetricsPublisher are available that can be configured as metrics publishers. C2 metrics are published through C2 specific properties, see [C2 documentation](C2.md) for more information on that.
 
-    # in minifi.properties
+The LogMetricsPublisher serializes all the configured metrics into a json output and writes the json to the MiNiFi logs periodically. LogMetricsPublisher follows the conventions of the C2 metrics, and all information is present in those metrics, including string data, is present in the log metrics as well. An example log entry may look like the following:

Review Comment:
   I think a 'that' is missing here:
   ```suggestion
   The LogMetricsPublisher serializes all the configured metrics into a json output and writes the json to the MiNiFi logs periodically. LogMetricsPublisher follows the conventions of the C2 metrics, and all information that is present in those metrics, including string data, is present in the log metrics as well. An example log entry may look like the following:
   ```



##########
libminifi/include/core/state/LogMetricsPublisher.h:
##########
@@ -0,0 +1,57 @@
+/**
+ *
+ * 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 <memory>
+#include <vector>
+#include <mutex>
+
+#include "core/logging/Logger.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "MetricsPublisher.h"
+#include "core/state/nodes/MetricsBase.h"
+#include "utils/LogUtils.h"
+#include "utils/StoppableThread.h"
+
+namespace org::apache::nifi::minifi::state {
+
+class LogMetricsPublisher : public MetricsPublisher {
+ public:
+  using MetricsPublisher::MetricsPublisher;
+
+  MINIFIAPI static constexpr const char* Description = "HTTP server that exposes MiNiFi metrics for Prometheus to scrape";

Review Comment:
   this Description needs to be updated



##########
METRICS.md:
##########
@@ -42,25 +42,66 @@ Aside from the publisher exposed metrics, metrics are also sent through C2 proto
 
 ## Configuration
 
-To configure the a metrics publisher first we have to set which publisher class should be used:
+Currently LogMetricsPublisher and PrometheusMetricsPublisher are available that can be configured as metrics publishers. C2 metrics are published through C2 specific properties, see [C2 documentation](C2.md) for more information on that.
 
-    # in minifi.properties
+The LogMetricsPublisher serializes all the configured metrics into a json output and writes the json to the MiNiFi logs periodically. LogMetricsPublisher follows the conventions of the C2 metrics, and all information is present in those metrics, including string data, is present in the log metrics as well. An example log entry may look like the following:
+
+    [2023-03-09 15:04:32.268] [org::apache::nifi::minifi::state::LogMetricsPublisher] [info] {"LogMetrics":{"RepositoryMetrics":{"flowfile":{"running":"true","full":"false","size":"0"},"provenance":{"running":"true","full":"false","size":"0"}}}}
 
-    nifi.metrics.publisher.class=PrometheusMetricsPublisher
+PrometheusMetricsPublisher publishes only numerical metrics to a Prometheus server in Prometheus specific format. This is different from the json format of the C2 and LogMetricsPublisher.
 
-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:
+### Common configuration properties
+
+To configure the a publisher first we have to specify the class in the properties. One or multiple publisher can be defined in comma separated format:
 
     # in minifi.properties
 
-    nifi.metrics.publisher.PrometheusMetricsPublisher.port=9936
+    nifi.metrics.publisher.class=LogMetricsPublisher
 
-The following option defines which metric classes should be exposed through the metrics publisher in configured with a comma separated value:
+    # alternatively
+
+    nifi.metrics.publisher.class=LogMetricsPublisher,PrometheusMetricsPublisher
+
+To define which metrics should be published either the generic or the publisher specific metrics property should be used. The generic metrics are applied to all publishers if no publisher specific metric is specified.
 
     # in minifi.properties
 
+    # define generic metrics for all selected publisher classes
+
     nifi.metrics.publisher.metrics=QueueMetrics,RepositoryMetrics,GetFileMetrics,DeviceInfoNode,FlowInformation,processorMetrics/Tail.*
 
+    # alternatively LogMetricsPublisher will only use the following metrics
+
+    nifi.metrics.publisher.LogMetricsPublisher.metrics=QueueMetrics,RepositoryMetrics
+
+Additional configuration properties may be required by specific publishers, these are listed below.
+
+### LogMetricsPublisher
+
+LogMetricsPublisher requires a logging interval to be configured which states how often the selected metrics should be logged
+
+    # in minifi.properties
+
+    # log the metrics in MiNiFi app logs every 30 seconds
+
+    nifi.metrics.publisher.LogMetricsPublisher.logging.interval=30s
+
+Optionally LogMetricsPublisher can be configured which log level should the publisher use. The default log level is INFO
+
+    # in minifi.properties
+
+    # change log level to debug
+
+    nifi.metrics.publisher.LogMetricsPublisher.log.level=DEBUG
+
+### PrometheusMetricsPublisher
+
+To use the PrometheusMetricsPublisher a port should also be configured where the metrics will be available to be scraped from:

Review Comment:
   'should be configured' is unclear.  If this is required, we should say so, like in line 81.  If it is optional, then we should say what the default port is.



##########
libminifi/include/core/state/LogMetricsPublisher.h:
##########
@@ -0,0 +1,57 @@
+/**
+ *
+ * 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 <memory>
+#include <vector>
+#include <mutex>
+
+#include "core/logging/Logger.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "MetricsPublisher.h"
+#include "core/state/nodes/MetricsBase.h"
+#include "utils/LogUtils.h"
+#include "utils/StoppableThread.h"
+
+namespace org::apache::nifi::minifi::state {
+
+class LogMetricsPublisher : public MetricsPublisher {
+ public:
+  using MetricsPublisher::MetricsPublisher;
+
+  MINIFIAPI static constexpr const char* Description = "HTTP server that exposes MiNiFi metrics for Prometheus to scrape";
+
+  void initialize(const std::shared_ptr<Configure>& configuration, const std::shared_ptr<state::response::ResponseNodeLoader>& response_node_loader) override;
+  void clearMetricNodes() override;
+  void loadMetricNodes() override;
+  ~LogMetricsPublisher();

Review Comment:
   the destructor should be marked `override`, as well



##########
libminifi/src/core/state/LogMetricsPublisher.cpp:
##########
@@ -0,0 +1,129 @@
+/**
+ *
+ * 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/LogMetricsPublisher.h"
+
+#include "core/Resource.h"
+#include "properties/Configuration.h"
+#include "core/TypedValues.h"
+
+namespace org::apache::nifi::minifi::state {
+
+LogMetricsPublisher::~LogMetricsPublisher() {
+  if (metrics_logger_thread_) {
+    metrics_logger_thread_->stopAndJoin();
+  }
+}
+
+void LogMetricsPublisher::initialize(const std::shared_ptr<Configure>& configuration, const std::shared_ptr<state::response::ResponseNodeLoader>& response_node_loader) {
+  state::MetricsPublisher::initialize(configuration, response_node_loader);
+  readLoggingInterval();
+  readLogLevel();
+}
+
+void LogMetricsPublisher::logMetrics() {
+  do {
+    response::SerializedResponseNode parent_node;
+    parent_node.name = "LogMetrics";
+    {
+      std::lock_guard<std::mutex> lock(response_node_mutex_);
+      for (const auto& response_node : response_nodes_) {
+        response::SerializedResponseNode metric_response_node;
+        metric_response_node.name = response_node->getName();
+        for (const auto& serialized_node : response_node->serialize()) {
+          metric_response_node.children.push_back(serialized_node);
+        }
+        parent_node.children.push_back(metric_response_node);
+      }
+    }
+    utils::LogUtils::logWithLevel(logger_, log_level_, parent_node.to_pretty_string().c_str());
+  } while (!utils::StoppableThread::waitForStopRequest(logging_interval_));
+}
+
+void LogMetricsPublisher::readLoggingInterval() {
+  gsl_Expects(configuration_);
+  if (auto logging_interval_str = configuration_->get(Configure::nifi_metrics_publisher_log_metrics_logging_interval)) {
+    if (auto logging_interval = minifi::core::TimePeriodValue::fromString(logging_interval_str.value())) {
+      logging_interval_ = logging_interval->getMilliseconds();
+      logger_->log_info("Metric logging interval is set to %lld milliseconds", logging_interval_.count());
+      return;
+    } else {
+      logger_->log_error("Configured logging interval is invalid!");

Review Comment:
   it would be useful to include `logging_interval_str.value()` in the error message



##########
libminifi/src/core/state/LogMetricsPublisher.cpp:
##########
@@ -0,0 +1,129 @@
+/**
+ *
+ * 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/LogMetricsPublisher.h"
+
+#include "core/Resource.h"
+#include "properties/Configuration.h"
+#include "core/TypedValues.h"
+
+namespace org::apache::nifi::minifi::state {
+
+LogMetricsPublisher::~LogMetricsPublisher() {
+  if (metrics_logger_thread_) {
+    metrics_logger_thread_->stopAndJoin();
+  }
+}
+
+void LogMetricsPublisher::initialize(const std::shared_ptr<Configure>& configuration, const std::shared_ptr<state::response::ResponseNodeLoader>& response_node_loader) {
+  state::MetricsPublisher::initialize(configuration, response_node_loader);
+  readLoggingInterval();
+  readLogLevel();
+}
+
+void LogMetricsPublisher::logMetrics() {
+  do {
+    response::SerializedResponseNode parent_node;
+    parent_node.name = "LogMetrics";
+    {
+      std::lock_guard<std::mutex> lock(response_node_mutex_);
+      for (const auto& response_node : response_nodes_) {
+        response::SerializedResponseNode metric_response_node;
+        metric_response_node.name = response_node->getName();
+        for (const auto& serialized_node : response_node->serialize()) {
+          metric_response_node.children.push_back(serialized_node);
+        }
+        parent_node.children.push_back(metric_response_node);
+      }
+    }
+    utils::LogUtils::logWithLevel(logger_, log_level_, parent_node.to_pretty_string().c_str());
+  } while (!utils::StoppableThread::waitForStopRequest(logging_interval_));
+}
+
+void LogMetricsPublisher::readLoggingInterval() {
+  gsl_Expects(configuration_);
+  if (auto logging_interval_str = configuration_->get(Configure::nifi_metrics_publisher_log_metrics_logging_interval)) {
+    if (auto logging_interval = minifi::core::TimePeriodValue::fromString(logging_interval_str.value())) {
+      logging_interval_ = logging_interval->getMilliseconds();
+      logger_->log_info("Metric logging interval is set to %lld milliseconds", logging_interval_.count());
+      return;
+    } else {
+      logger_->log_error("Configured logging interval is invalid!");
+    }
+  }
+
+  throw Exception(GENERAL_EXCEPTION, "Metrics logging interval not configured for log metrics publisher!");
+}
+
+void LogMetricsPublisher::readLogLevel() {
+  gsl_Expects(configuration_);
+  if (auto log_level_str = configuration_->get(Configure::nifi_metrics_publisher_log_metrics_log_level)) {
+    std::transform(log_level_str->begin(), log_level_str->end(), log_level_str->begin(), ::toupper);

Review Comment:
   Please use `StringUtils::toUpper()`, as calling `::toupper` with a negative (signed) `char` argument is technically UB.
   
   So much code out there does this "incorrectly" that I don't think it really matters in practice, but since we already have the helper function in `StringUtils`, we might as well use it.



##########
libminifi/include/core/state/LogMetricsPublisher.h:
##########
@@ -0,0 +1,57 @@
+/**
+ *
+ * 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 <memory>
+#include <vector>
+#include <mutex>
+
+#include "core/logging/Logger.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "MetricsPublisher.h"
+#include "core/state/nodes/MetricsBase.h"
+#include "utils/LogUtils.h"
+#include "utils/StoppableThread.h"
+
+namespace org::apache::nifi::minifi::state {
+
+class LogMetricsPublisher : public MetricsPublisher {
+ public:
+  using MetricsPublisher::MetricsPublisher;
+
+  MINIFIAPI static constexpr const char* Description = "HTTP server that exposes MiNiFi metrics for Prometheus to scrape";
+
+  void initialize(const std::shared_ptr<Configure>& configuration, const std::shared_ptr<state::response::ResponseNodeLoader>& response_node_loader) override;
+  void clearMetricNodes() override;
+  void loadMetricNodes() override;
+  ~LogMetricsPublisher();
+
+ private:
+  void readLoggingInterval();
+  void readLogLevel();
+  void logMetrics();
+
+  std::unique_ptr<utils::StoppableThread> metrics_logger_thread_;
+  utils::LogUtils::LogLevelOption log_level_ = utils::LogUtils::LogLevelOption::LOGGING_INFO;
+  std::chrono::milliseconds logging_interval_;
+  std::mutex response_node_mutex_;

Review Comment:
   very minor, but I think `response_nodes_mutex_` would be a better name, since it guards `response_nodes_`, not `response_node_`



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