Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -2123,6 +2123,10 @@ public String getMetricReporterMetricsNamePrefix() {
return getStringOrDefault(HoodieMetricsConfig.METRICS_REPORTER_PREFIX);
}

public String getMetricReporterFileBasedConfigs() {
return getStringOrDefault(HoodieMetricsConfig.METRICS_REPORTER_FILE_BASED_CONFIGS_PATH);
}

/**
* memory configs.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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.14.0")
.withDocumentation("Comma separated list of config file paths for metric exporter configs");

/**
* @deprecated Use {@link #TURN_METRICS_ON} and its methods instead
*/
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,81 @@
/*
* 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.StringUtils;
import org.apache.hudi.common.util.ValidationUtils;
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 final class MetricUtils {

// Example metric:- with_label_metric;group:a,job:0. Here `with_label_metric` is the metric name.
// `group:a` and `job:0` are the labels for this metric.
// Metric name and labels are separated by `;`
private static final String METRIC_NAME_AND_LABELS_SEPARATOR = ";";
// Multiple Labels are separated by `,`
private static final String LABELS_SEPARATOR = ",";
// Label key and value is separated by `:`
private static final String LABELS_KEY_AND_VALUE_SEPARATOR = ":";

private static Pair<String, String> splitToPair(String label) {
String[] keyValues = label.split(LABELS_KEY_AND_VALUE_SEPARATOR, 2);
ValidationUtils.checkArgument(StringUtils.nonEmpty(keyValues[0]), String.format("Key is empty for label %s", label));
return Pair.of(keyValues[0], keyValues.length == 2 ? keyValues[1] : "");
}

public static Pair<String,Map<String, String>> getLabelsAndMetricMap(String metric) {
Pair<String, List<String>> labelsList = getLabelsAndMetricList(metric);
return Pair.of(labelsList.getLeft(), getLabelsAsMap(labelsList.getValue()));
}

public static Pair<String,String> getMetricAndLabels(String metric) {
String[] tokens = metric.split(METRIC_NAME_AND_LABELS_SEPARATOR);
if (tokens.length > 2) {
throw new RuntimeException("more than one ';' detected in metric string");
}
if (tokens.length == 2) {
return Pair.of(tokens[0], tokens[1]);
}
return Pair.of(tokens[0], "");
}

public static Map<String, String> getLabelsAsMap(String labels) {
return getLabelsAsMap(getLabels(labels));
}

public static Map<String, String> getLabelsAsMap(List<String> labels) {
return labels.stream().filter(StringUtils::nonEmpty).map(MetricUtils::splitToPair)
.collect(Collectors.toMap(Pair::getLeft, Pair::getRight, (v1, v2) -> {
throw new IllegalStateException(String.format("Multiple values {%s, %s} for same key", v1, v2));
}));
}

public static List<String> getLabels(String labels) {
return Arrays.stream(labels.split(LABELS_SEPARATOR)).filter(StringUtils::nonEmpty).collect(Collectors.toList());
}

public static Pair<String,List<String>> getLabelsAndMetricList(String metric) {
Pair<String, String> metricAndLabels = getMetricAndLabels(metric);
return Pair.of(metricAndLabels.getLeft(), getLabels(metricAndLabels.getRight()));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,15 +18,23 @@

package org.apache.hudi.metrics;

import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.metrics.Registry;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.config.HoodieWriteConfig;

import com.codahale.metrics.MetricRegistry;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;

import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;

/**
Expand All @@ -39,18 +47,23 @@ public class Metrics {
private static final Map<String, Metrics> METRICS_INSTANCE_PER_BASEPATH = new HashMap<>();

private final MetricRegistry registry;
private MetricsReporter reporter;
private final List<MetricsReporter> reporters;
private final String commonMetricPrefix;
private boolean initialized = false;

public 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<>();
Option<MetricsReporter> defaultReporter = MetricsReporterFactory.createReporter(metricConfig, registry);
defaultReporter.ifPresent(reporters::add);
if (StringUtils.nonEmpty(metricConfig.getMetricReporterFileBasedConfigs())) {
reporters.addAll(addAdditionalMetricsExporters(metricConfig));
}
reporter.start();
if (reporters.size() == 0) {
throw new RuntimeException("Cannot initialize Reporters.");
}
reporters.forEach(MetricsReporter::start);

Runtime.getRuntime().addShutdownHook(new Thread(this::shutdown));
this.initialized = true;
Expand All @@ -75,12 +88,34 @@ public static synchronized void shutdownAllMetrics() {
METRICS_INSTANCE_PER_BASEPATH.values().forEach(Metrics::shutdown);
}

private List<MetricsReporter> addAdditionalMetricsExporters(HoodieWriteConfig metricConfig) {
List<MetricsReporter> reporterList = new ArrayList<>();
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();
Option<MetricsReporter> reporter = MetricsReporterFactory.createReporter(secondarySourceConfig, registry);
if (reporter.isPresent()) {
reporterList.add(reporter.get());
} else {
LOG.error(String.format("Could not create reporter using properties path %s base path %s",
propPath, metricConfig.getBasePath()));
}
}
} catch (IOException e) {
LOG.error("Failed to add MetricsExporters", e);
}
LOG.info("total additional metrics reporters added =" + reporterList.size());
return reporterList;
}

public synchronized void shutdown() {
try {
registerHoodieCommonMetrics();
reporter.report();
reporters.forEach(MetricsReporter::report);
LOG.info("Stopping the metrics reporter...");
reporter.stop();
reporters.forEach(MetricsReporter::stop);
} catch (Exception e) {
LOG.warn("Error while closing reporter", e);
} finally {
Expand All @@ -92,13 +127,14 @@ public synchronized void flush() {
try {
LOG.info("Reporting and flushing all metrics");
registerHoodieCommonMetrics();
reporter.report();
reporters.forEach(MetricsReporter::report);
registry.getNames().forEach(this.registry::remove);
registerHoodieCommonMetrics();
} catch (Exception e) {
LOG.error("Error while reporting and flushing metrics", e);
}
}

public void registerGauges(Map<String, Long> metricsMap, Option<String> prefix) {
String metricPrefix = prefix.isPresent() ? prefix.get() + "." : "";
metricsMap.forEach((k, v) -> registerGauge(metricPrefix + k, v));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,9 +18,11 @@

package org.apache.hudi.metrics;

import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.config.metrics.HoodieMetricsConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.metrics.cloudwatch.CloudWatchMetricsReporter;
import org.apache.hudi.metrics.custom.CustomizableMetricsReporter;
Expand All @@ -41,7 +43,7 @@ public class MetricsReporterFactory {

private static final Logger LOG = LogManager.getLogger(MetricsReporterFactory.class);

public static MetricsReporter createReporter(HoodieWriteConfig config, MetricRegistry registry) {
public static Option<MetricsReporter> createReporter(HoodieWriteConfig config, MetricRegistry registry) {
String reporterClassName = config.getMetricReporterClassName();

if (!StringUtils.isNullOrEmpty(reporterClassName)) {
Expand All @@ -51,11 +53,17 @@ public static MetricsReporter createReporter(HoodieWriteConfig config, MetricReg
throw new HoodieException(config.getMetricReporterClassName()
+ " is not a subclass of CustomizableMetricsReporter");
}
return (MetricsReporter) instance;
return Option.of((MetricsReporter) instance);
}

MetricsReporterType type = config.getMetricsReporterType();
MetricsReporter reporter = null;
if (type == null) {
LOG.warn(String.format("Metric creation failed. %s is not configured",
HoodieMetricsConfig.METRICS_REPORTER_TYPE_VALUE.key()));
return Option.empty();
}

switch (type) {
case GRAPHITE:
reporter = new MetricsGraphiteReporter(config, registry);
Expand Down Expand Up @@ -85,6 +93,6 @@ public static MetricsReporter createReporter(HoodieWriteConfig config, MetricReg
LOG.error("Reporter type[" + type + "] is not supported.");
break;
}
return reporter;
return Option.ofNullable(reporter);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.hudi.metrics.datadog;

import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.common.util.ValidationUtils;

Expand Down Expand Up @@ -47,32 +48,37 @@ public class DatadogHttpClient implements Closeable {

private static final Logger LOG = LogManager.getLogger(DatadogHttpClient.class);

private static final String SERIES_URL_FORMAT = "https://app.datadoghq.%s/api/v1/series";
private static final String VALIDATE_URL_FORMAT = "https://app.datadoghq.%s/api/v1/validate";
private static final String DEFAULT_HOST = "app.us.datadoghq";
private static final String SERIES_URL_FORMAT = "https://%s.%s/api/v1/series";
private static final String VALIDATE_URL_FORMAT = "https://%s.%s/api/v1/validate";
private static final String HEADER_KEY_API_KEY = "DD-API-KEY";

private final String apiKey;
private final String seriesUrl;
private final String validateUrl;
private final CloseableHttpClient client;

public DatadogHttpClient(ApiSite apiSite, String apiKey, boolean skipValidation, CloseableHttpClient client) {
public DatadogHttpClient(ApiSite apiSite, String apiKey, boolean skipValidation, CloseableHttpClient client, Option<String> host) {
this.apiKey = apiKey;
this.seriesUrl = String.format(SERIES_URL_FORMAT, apiSite.getDomain());
this.validateUrl = String.format(VALIDATE_URL_FORMAT, apiSite.getDomain());
this.seriesUrl = String.format(SERIES_URL_FORMAT, host.orElse(DEFAULT_HOST), apiSite.getDomain());
this.validateUrl = String.format(VALIDATE_URL_FORMAT, host.orElse(DEFAULT_HOST), apiSite.getDomain());
this.client = client;
if (!skipValidation) {
validateApiKey();
}
}

public DatadogHttpClient(ApiSite apiSite, String apiKey, boolean skipValidation, int timeoutSeconds) {
public DatadogHttpClient(ApiSite apiSite, String apiKey, boolean skipValidation, CloseableHttpClient client) {
this(apiSite, apiKey, skipValidation, client, Option.of(DEFAULT_HOST));
}

public DatadogHttpClient(ApiSite apiSite, String apiKey, boolean skipValidation, int timeoutSeconds, Option<String> host) {
this(apiSite, apiKey, skipValidation, HttpClientBuilder.create()
.setDefaultRequestConfig(RequestConfig.custom()
.setConnectTimeout(timeoutSeconds * 1000)
.setConnectionRequestTimeout(timeoutSeconds * 1000)
.setSocketTimeout(timeoutSeconds * 1000).build())
.build());
.build(), host);
}

private void validateApiKey() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ public DatadogMetricsReporter(HoodieWriteConfig config, MetricRegistry registry)

reporter = new DatadogReporter(
registry,
new DatadogHttpClient(apiSite, apiKey, skipValidation, timeoutSeconds),
new DatadogHttpClient(apiSite, apiKey, skipValidation, timeoutSeconds, host),
prefix,
host,
tags,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,8 @@

import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.metrics.MetricUtils;

import com.codahale.metrics.Clock;
import com.codahale.metrics.Counter;
Expand All @@ -39,6 +41,7 @@

import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.SortedMap;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
Expand Down Expand Up @@ -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()));
host.ifPresent(builder::withHost);
List<String> runTimeLables = labelsKeyValue.getKey();
tags.map(runTimeLables::addAll);
builder.withTags(runTimeLables);
client.send(builder.build());
});

host.ifPresent(builder::withHost);
tags.ifPresent(builder::withTags);

client.send(builder.build());
}

protected String prefix(String... components) {
Expand Down
Loading