codope commented on code in PR #8041:
URL: https://github.com/apache/hudi/pull/8041#discussion_r1118356753


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/Metrics.java:
##########
@@ -40,27 +48,51 @@ public class Metrics {
   private static Metrics instance = null;
 
   private final MetricRegistry registry;
-  private MetricsReporter reporter;
+  private final List<MetricsReporter> reporters;
   private final String commonMetricPrefix;
 
   private Metrics(HoodieWriteConfig metricConfig) {
     registry = new MetricRegistry();
     commonMetricPrefix = metricConfig.getMetricReporterMetricsNamePrefix();
-    reporter = MetricsReporterFactory.createReporter(metricConfig, registry);
-    if (reporter == null) {
-      throw new RuntimeException("Cannot initialize Reporter.");
+    reporters = new ArrayList<>();
+    MetricsReporter defaultReporter = 
MetricsReporterFactory.createReporter(metricConfig, registry);
+    if (defaultReporter != null) {
+      reporters.add(defaultReporter);
     }
-    reporter.start();
+    reporters.addAll(addAdditionalMetricsExporters(metricConfig));
+    if (reporters.size() == 0) {
+      throw new RuntimeException("Cannot initialize Reporters.");
+    }
+    reporters.forEach(MetricsReporter::start);
 
     Runtime.getRuntime().addShutdownHook(new Thread(Metrics::shutdown));
   }
 
+  private List<MetricsReporter> 
addAdditionalMetricsExporters(HoodieWriteConfig metricConfig) {
+    List<MetricsReporter> reporterList = new ArrayList<>();
+    if 
(!StringUtils.isNullOrEmpty(metricConfig.getMetricReporterFileBasedConfigs())) {

Review Comment:
   move this check to constructor. it's better to not add empty list



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/Metrics.java:
##########
@@ -40,27 +48,51 @@ public class Metrics {
   private static Metrics instance = null;
 
   private final MetricRegistry registry;
-  private MetricsReporter reporter;
+  private final List<MetricsReporter> reporters;
   private final String commonMetricPrefix;
 
   private Metrics(HoodieWriteConfig metricConfig) {
     registry = new MetricRegistry();
     commonMetricPrefix = metricConfig.getMetricReporterMetricsNamePrefix();
-    reporter = MetricsReporterFactory.createReporter(metricConfig, registry);
-    if (reporter == null) {
-      throw new RuntimeException("Cannot initialize Reporter.");
+    reporters = new ArrayList<>();
+    MetricsReporter defaultReporter = 
MetricsReporterFactory.createReporter(metricConfig, registry);
+    if (defaultReporter != null) {
+      reporters.add(defaultReporter);
     }
-    reporter.start();
+    reporters.addAll(addAdditionalMetricsExporters(metricConfig));
+    if (reporters.size() == 0) {
+      throw new RuntimeException("Cannot initialize Reporters.");
+    }
+    reporters.forEach(MetricsReporter::start);
 
     Runtime.getRuntime().addShutdownHook(new Thread(Metrics::shutdown));
   }
 
+  private List<MetricsReporter> 
addAdditionalMetricsExporters(HoodieWriteConfig metricConfig) {
+    List<MetricsReporter> reporterList = new ArrayList<>();
+    if 
(!StringUtils.isNullOrEmpty(metricConfig.getMetricReporterFileBasedConfigs())) {
+      List<String> propPathList = 
StringUtils.split(metricConfig.getMetricReporterFileBasedConfigs(), ",");
+      try (FileSystem fs = FSUtils.getFs(propPathList.get(0), new 
Configuration())) {
+        for (String propPath: propPathList) {
+          HoodieWriteConfig secondarySourceConfig = 
HoodieWriteConfig.newBuilder().fromInputStream(
+              fs.open(new 
Path(propPath))).withPath(metricConfig.getBasePath()).build();
+          
reporterList.add(MetricsReporterFactory.createReporter(secondarySourceConfig, 
registry));
+        }
+      } catch (IOException e) {
+        LOG.error("Failed to add MetricsExporters", e);

Review Comment:
   let's remove this redundant log? the exception below already includes it.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricUtils.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.
+ */
+
+package org.apache.hudi.metrics;
+
+import org.apache.hudi.common.util.collection.Pair;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class MetricUtils {
+
+  private static Pair<String, String> splitToPair(String label) {
+    String[] keyValuess = label.split(":");

Review Comment:
   split can return empty string.. does it make sense to have empty key and 
value in that case?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricUtils.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.
+ */
+
+package org.apache.hudi.metrics;
+
+import org.apache.hudi.common.util.collection.Pair;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class MetricUtils {
+
+  private static Pair<String, String> splitToPair(String label) {
+    String[] keyValuess = label.split(":");

Review Comment:
   ```suggestion
       String[] keyValues = label.split(":");
   ```



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/datadog/DatadogReporter.java:
##########
@@ -84,18 +87,22 @@ public void report(
       SortedMap<String, Histogram> histograms,
       SortedMap<String, Meter> meters,
       SortedMap<String, Timer> timers) {
-    final long now = clock.getTime() / 1000;
-    final PayloadBuilder builder = new PayloadBuilder();
 
-    builder.withMetricType(MetricType.gauge);
-    gauges.forEach((metricName, metric) -> {
-      builder.addGauge(prefix(metricName), now, (long) metric.getValue());
+    Map<List<String>, List<Pair<String, List<String>>>> labelsPair = 
gauges.keySet().stream().map(MetricUtils::getLabelsAndMetricList)
+        .collect(Collectors.groupingBy(Pair::getValue));
+    labelsPair.entrySet().forEach(labelsKeyValue -> {
+      final long now = clock.getTime() / 1000;
+      final PayloadBuilder builder = new PayloadBuilder();
+      builder.withMetricType(MetricType.gauge);
+      gauges.forEach((metricName, metric) -> {
+        
builder.addGauge(prefix(MetricUtils.getMetricAndLabels(metricName).getKey()), 
now, (long) metric.getValue());
+      });

Review Comment:
   ```suggestion
         gauges.forEach((metricName, metric) -> 
builder.addGauge(prefix(MetricUtils.getMetricAndLabels(metricName).getKey()), 
now, (long) metric.getValue()));
   ```



##########
hudi-client/hudi-client-common/src/test/resources/datadog.properties:
##########
@@ -0,0 +1,25 @@
+###
+# 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.
+###
+hoodie.base.path="/tmp/base_path"
+hoodie.metrics.datadog.api.key=f4e1ca9f6e3f16ff5c0800dddfda6179
+hoodie.metrics.datadog.report.period.seconds=3600
+hoodie.metrics.datadog.job.name=test_job
+hoodie.metrics.datadog.metric.host=app.us5.datadoghq
+hoodie.metrics.datadog.metric.prefix=oh
+hoodie.metrics.datadog.api.site=US
+hoodie.metrics.reporter.type=DATADOG

Review Comment:
   nit: add a new line in both the properties files.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/Metrics.java:
##########
@@ -118,6 +150,11 @@ public static void registerGauges(Map<String, Long> 
metricsMap, Option<String> p
     metricsMap.forEach((k, v) -> registerGauge(metricPrefix + k, v));
   }
 
+  public static void registerGauges(Map<String, Long> metricsMap, 
Option<String> prefix, Map<String, String> labels) {

Review Comment:
   There are two `registerGauges` and this one is not getting used anywhere 
other than the tests. Exposing a public method just for tests does is not a 
good practice. Is there a better way to refactor?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/Metrics.java:
##########
@@ -40,27 +48,51 @@ public class Metrics {
   private static Metrics instance = null;
 
   private final MetricRegistry registry;
-  private MetricsReporter reporter;
+  private final List<MetricsReporter> reporters;
   private final String commonMetricPrefix;
 
   private Metrics(HoodieWriteConfig metricConfig) {
     registry = new MetricRegistry();
     commonMetricPrefix = metricConfig.getMetricReporterMetricsNamePrefix();
-    reporter = MetricsReporterFactory.createReporter(metricConfig, registry);
-    if (reporter == null) {
-      throw new RuntimeException("Cannot initialize Reporter.");
+    reporters = new ArrayList<>();
+    MetricsReporter defaultReporter = 
MetricsReporterFactory.createReporter(metricConfig, registry);
+    if (defaultReporter != null) {
+      reporters.add(defaultReporter);
     }
-    reporter.start();
+    reporters.addAll(addAdditionalMetricsExporters(metricConfig));
+    if (reporters.size() == 0) {
+      throw new RuntimeException("Cannot initialize Reporters.");
+    }
+    reporters.forEach(MetricsReporter::start);
 
     Runtime.getRuntime().addShutdownHook(new Thread(Metrics::shutdown));
   }
 
+  private List<MetricsReporter> 
addAdditionalMetricsExporters(HoodieWriteConfig metricConfig) {
+    List<MetricsReporter> reporterList = new ArrayList<>();
+    if 
(!StringUtils.isNullOrEmpty(metricConfig.getMetricReporterFileBasedConfigs())) {

Review Comment:
   There is also a method `StringUtils#nonEmpty`



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/metrics/HoodieMetricsConfig.java:
##########
@@ -95,6 +95,12 @@ public class HoodieMetricsConfig extends HoodieConfig {
       .sinceVersion("0.13.0")
       .withDocumentation("Enable metrics for locking infra. Useful when 
operating in multiwriter mode");
 
+  public static final ConfigProperty<String> 
METRICS_REPORTER_FILE_BASED_CONFIGS_PATH = ConfigProperty
+      .key(METRIC_PREFIX + ".configs.properties")
+      .defaultValue("")
+      .sinceVersion("0.12.2")

Review Comment:
   this should be 0.14.0 right?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/Metrics.java:
##########
@@ -40,27 +48,51 @@ public class Metrics {
   private static Metrics instance = null;
 
   private final MetricRegistry registry;
-  private MetricsReporter reporter;
+  private final List<MetricsReporter> reporters;
   private final String commonMetricPrefix;
 
   private Metrics(HoodieWriteConfig metricConfig) {
     registry = new MetricRegistry();
     commonMetricPrefix = metricConfig.getMetricReporterMetricsNamePrefix();
-    reporter = MetricsReporterFactory.createReporter(metricConfig, registry);
-    if (reporter == null) {
-      throw new RuntimeException("Cannot initialize Reporter.");
+    reporters = new ArrayList<>();
+    MetricsReporter defaultReporter = 
MetricsReporterFactory.createReporter(metricConfig, registry);
+    if (defaultReporter != null) {
+      reporters.add(defaultReporter);
     }
-    reporter.start();
+    reporters.addAll(addAdditionalMetricsExporters(metricConfig));
+    if (reporters.size() == 0) {
+      throw new RuntimeException("Cannot initialize Reporters.");
+    }
+    reporters.forEach(MetricsReporter::start);
 
     Runtime.getRuntime().addShutdownHook(new Thread(Metrics::shutdown));
   }
 
+  private List<MetricsReporter> 
addAdditionalMetricsExporters(HoodieWriteConfig metricConfig) {
+    List<MetricsReporter> reporterList = new ArrayList<>();
+    if 
(!StringUtils.isNullOrEmpty(metricConfig.getMetricReporterFileBasedConfigs())) {
+      List<String> propPathList = 
StringUtils.split(metricConfig.getMetricReporterFileBasedConfigs(), ",");
+      try (FileSystem fs = FSUtils.getFs(propPathList.get(0), new 
Configuration())) {
+        for (String propPath: propPathList) {
+          HoodieWriteConfig secondarySourceConfig = 
HoodieWriteConfig.newBuilder().fromInputStream(
+              fs.open(new 
Path(propPath))).withPath(metricConfig.getBasePath()).build();
+          
reporterList.add(MetricsReporterFactory.createReporter(secondarySourceConfig, 
registry));
+        }
+      } catch (IOException e) {
+        LOG.error("Failed to add MetricsExporters", e);
+        throw new HoodieException("failed to MetricsExporters", e);

Review Comment:
   i'm wondering whether we should throw an exception of just log a warning. I 
think failing the deltastreamer because one of the metrics exporters could not 
be registered is a bit aggressive. Typically, upstream systems will have some 
monitor over missing metrics. So, can we let the deltasync continue with just a 
warning?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsReporterFactory.java:
##########
@@ -56,6 +56,9 @@ public static MetricsReporter 
createReporter(HoodieWriteConfig config, MetricReg
 
     MetricsReporterType type = config.getMetricsReporterType();
     MetricsReporter reporter = null;
+    if (type == null) {
+      return reporter;

Review Comment:
   this can return `null` reporter and we have to be extra careful at the 
caller, e.g. when it is called to add to the list in 
`addAdditionalMetricsExporters` it will throw and exception. It's better to 
handle null here itself.



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to