ryucc commented on a change in pull request #1558: URL: https://github.com/apache/samza/pull/1558#discussion_r748824201
########## File path: samza-core/src/main/java/org/apache/samza/metrics/reporter/Metrics.java ########## @@ -0,0 +1,79 @@ +/* + * 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.samza.metrics.reporter; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + + +public class Metrics { + private final Map<String, Map<String, Object>> immutableMetrics = new HashMap<>(); + + public Metrics() { + this(Collections.emptyMap()); + } + + public Metrics(Map<String, Map<String, Object>> metrics) { + metrics.forEach((groupKey, groupValue) -> { + Map<String, Object> immutableMetricGroup = new HashMap<>(); + groupValue.forEach(immutableMetricGroup::put); Review comment: Is this line `immutableMetricGroup.putAll(groupValue)`? Or just `Map<String, Object> immutableMetricGroup = groupValue.clone()`? Don't think this is a deep clone either way. ########## File path: samza-core/src/main/java/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.java ########## @@ -0,0 +1,228 @@ +/* + * 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.samza.metrics.reporter; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.samza.SamzaException; +import org.apache.samza.config.ShellCommandConfig; +import org.apache.samza.metrics.Counter; +import org.apache.samza.metrics.Gauge; +import org.apache.samza.metrics.MetricsReporter; +import org.apache.samza.metrics.MetricsVisitor; +import org.apache.samza.metrics.ReadableMetricsRegistry; +import org.apache.samza.metrics.RegistryWithSource; +import org.apache.samza.metrics.Timer; +import org.apache.samza.serializers.Serializer; +import org.apache.samza.system.OutgoingMessageEnvelope; +import org.apache.samza.system.SystemProducer; +import org.apache.samza.system.SystemStream; +import org.apache.samza.util.Clock; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * MetricsSnapshotReporter is a generic metrics reporter that sends metrics to a stream. + * + * jobName // my-samza-job + * jobId // an id that differentiates multiple executions of the same job + * taskName // container_567890 + * host // eat1-app128.gird + * version // 0.0.1 + * blacklist // Regex of metrics to ignore when flushing + */ +public class MetricsSnapshotReporter implements MetricsReporter, Runnable { + private static final Logger LOG = LoggerFactory.getLogger(MetricsSnapshotReporter.class); + + private final SystemProducer producer; + private final SystemStream out; + private final int reportingInterval; + private final String jobName; + private final String jobId; + private final String containerName; + private final String version; + private final String samzaVersion; + private final String host; + private final Serializer<MetricsSnapshot> serializer; + private final Optional<String> blacklist; + private final Clock clock; + + private final String execEnvironmentContainerId; + private final ScheduledExecutorService executor; + private final long resetTime; + private final List<RegistryWithSource> registries = new ArrayList<>(); + private final Set<String> blacklistedMetrics = new HashSet<>(); + + public MetricsSnapshotReporter(SystemProducer producer, SystemStream out, int reportingInterval, String jobName, + String jobId, String containerName, String version, String samzaVersion, String host, + Serializer<MetricsSnapshot> serializer, Optional<String> blacklist, Clock clock) { + this.producer = producer; + this.out = out; + this.reportingInterval = reportingInterval; + this.jobName = jobName; + this.jobId = jobId; + this.containerName = containerName; + this.version = version; + this.samzaVersion = samzaVersion; + this.host = host; + this.serializer = serializer; + this.blacklist = blacklist; + this.clock = clock; + + this.execEnvironmentContainerId = + Optional.ofNullable(System.getenv(ShellCommandConfig.ENV_EXECUTION_ENV_CONTAINER_ID)).orElse(""); + this.executor = Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder().setNameFormat("Samza MetricsSnapshotReporter Thread-%d").setDaemon(true).build()); + this.resetTime = this.clock.currentTimeMillis(); + LOG.info( + "got metrics snapshot reporter properties [job name: {}, job id: {}, containerName: {}, version: {}, samzaVersion: {}, host: {}, reportingInterval {}]", + jobName, jobId, containerName, version, samzaVersion, host, reportingInterval); + } + + @Override + public void start() { + LOG.info("Starting producer."); + this.producer.start(); + LOG.info("Starting reporter timer."); + this.executor.scheduleWithFixedDelay(this, 0, reportingInterval, TimeUnit.SECONDS); + } + + @Override + public void register(String source, ReadableMetricsRegistry registry) { + this.registries.add(new RegistryWithSource(source, registry)); + LOG.info("Registering {} with producer.", source); + this.producer.register(source); + } + + @Override + public void stop() { + // Scheduling an event with 0 delay to ensure flushing of metrics one last time before shutdown + this.executor.schedule(this, 0, TimeUnit.SECONDS); + LOG.info("Stopping reporter timer."); + // Allow the scheduled task above to finish, and block for termination (for max 60 seconds) + this.executor.shutdown(); + try { + this.executor.awaitTermination(60, TimeUnit.SECONDS); + } catch (InterruptedException e) { + throw new SamzaException(e); + } + LOG.info("Stopping producer."); + this.producer.stop(); + if (!this.executor.isTerminated()) { + LOG.warn("Unable to shutdown reporter timer."); + } + } + + @Override + public void run() { + try { + innerRun(); + } catch (Exception e) { + // Ignore all exceptions - because subsequent executions of this scheduled task will be suppressed + // by the executor if the current task throws an unhandled exception. + LOG.warn("Error while reporting metrics. Will retry in " + reportingInterval + " seconds.", e); + } + } + + public void innerRun() { + LOG.debug("Begin flushing metrics."); + for (RegistryWithSource registryWithSource : this.registries) { + String source = registryWithSource.getSource(); + ReadableMetricsRegistry registry = registryWithSource.getRegistry(); + LOG.debug("Flushing metrics for {}.", source); + Map<String, Map<String, Object>> metricsMsg = new HashMap<>(); + + // metrics + registry.getGroups().forEach(group -> { + Map<String, Object> groupMsg = new HashMap<>(); + registry.getGroup(group).forEach((name, metric) -> { + if (!shouldIgnore(group, name)) { + metric.visit(new MetricsVisitor() { + @Override + public void counter(Counter counter) { + groupMsg.put(name, counter.getCount()); + } + + @Override + public <T> void gauge(Gauge<T> gauge) { + groupMsg.put(name, gauge.getValue()); + } + + @Override + public void timer(Timer timer) { + groupMsg.put(name, timer.getSnapshot().getAverage()); + } + }); + } + }); + + // dont emit empty groups + if (!groupMsg.isEmpty()) { + metricsMsg.put(group, groupMsg); + } + }); + + // publish to Kafka only if the metricsMsg carries any metrics + if (!metricsMsg.isEmpty()) { + MetricsHeader header = + new MetricsHeader(this.jobName, this.jobId, this.containerName, this.execEnvironmentContainerId, source, + this.version, this.samzaVersion, this.host, this.clock.currentTimeMillis(), this.resetTime); + Metrics metrics = new Metrics(metricsMsg); + LOG.debug("Flushing metrics for {} to {} with header and map: header={}, map={}.", source, out, + header.getAsMap(), metrics.getAsMap()); + MetricsSnapshot metricsSnapshot = new MetricsSnapshot(header, metrics); + Object maybeSerialized = (this.serializer != null) ? this.serializer.toBytes(metricsSnapshot) : metricsSnapshot; + try { + this.producer.send(source, new OutgoingMessageEnvelope(this.out, this.host, null, maybeSerialized)); + // Always flush, since we don't want metrics to get batched up. + this.producer.flush(source); + } catch (Exception e) { + LOG.error(String.format("Exception when flushing metrics for source %s", source), e); + } + } + } + LOG.debug("Finished flushing metrics."); + } + + protected boolean shouldIgnore(String group, String metricName) { + boolean isBlacklisted = this.blacklist.isPresent(); Review comment: Can return early if false? This might reduce 1 layer of the nested if. ########## File path: samza-core/src/main/java/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.java ########## @@ -0,0 +1,228 @@ +/* + * 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.samza.metrics.reporter; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.samza.SamzaException; +import org.apache.samza.config.ShellCommandConfig; +import org.apache.samza.metrics.Counter; +import org.apache.samza.metrics.Gauge; +import org.apache.samza.metrics.MetricsReporter; +import org.apache.samza.metrics.MetricsVisitor; +import org.apache.samza.metrics.ReadableMetricsRegistry; +import org.apache.samza.metrics.RegistryWithSource; +import org.apache.samza.metrics.Timer; +import org.apache.samza.serializers.Serializer; +import org.apache.samza.system.OutgoingMessageEnvelope; +import org.apache.samza.system.SystemProducer; +import org.apache.samza.system.SystemStream; +import org.apache.samza.util.Clock; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * MetricsSnapshotReporter is a generic metrics reporter that sends metrics to a stream. + * + * jobName // my-samza-job + * jobId // an id that differentiates multiple executions of the same job + * taskName // container_567890 + * host // eat1-app128.gird + * version // 0.0.1 + * blacklist // Regex of metrics to ignore when flushing + */ +public class MetricsSnapshotReporter implements MetricsReporter, Runnable { + private static final Logger LOG = LoggerFactory.getLogger(MetricsSnapshotReporter.class); + + private final SystemProducer producer; + private final SystemStream out; + private final int reportingInterval; + private final String jobName; + private final String jobId; + private final String containerName; + private final String version; + private final String samzaVersion; + private final String host; + private final Serializer<MetricsSnapshot> serializer; + private final Optional<String> blacklist; + private final Clock clock; + + private final String execEnvironmentContainerId; + private final ScheduledExecutorService executor; + private final long resetTime; + private final List<RegistryWithSource> registries = new ArrayList<>(); + private final Set<String> blacklistedMetrics = new HashSet<>(); + + public MetricsSnapshotReporter(SystemProducer producer, SystemStream out, int reportingInterval, String jobName, + String jobId, String containerName, String version, String samzaVersion, String host, + Serializer<MetricsSnapshot> serializer, Optional<String> blacklist, Clock clock) { + this.producer = producer; + this.out = out; + this.reportingInterval = reportingInterval; + this.jobName = jobName; + this.jobId = jobId; + this.containerName = containerName; + this.version = version; + this.samzaVersion = samzaVersion; + this.host = host; + this.serializer = serializer; + this.blacklist = blacklist; + this.clock = clock; + + this.execEnvironmentContainerId = + Optional.ofNullable(System.getenv(ShellCommandConfig.ENV_EXECUTION_ENV_CONTAINER_ID)).orElse(""); + this.executor = Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder().setNameFormat("Samza MetricsSnapshotReporter Thread-%d").setDaemon(true).build()); + this.resetTime = this.clock.currentTimeMillis(); + LOG.info( + "got metrics snapshot reporter properties [job name: {}, job id: {}, containerName: {}, version: {}, samzaVersion: {}, host: {}, reportingInterval {}]", + jobName, jobId, containerName, version, samzaVersion, host, reportingInterval); + } + + @Override + public void start() { + LOG.info("Starting producer."); + this.producer.start(); + LOG.info("Starting reporter timer."); + this.executor.scheduleWithFixedDelay(this, 0, reportingInterval, TimeUnit.SECONDS); + } + + @Override + public void register(String source, ReadableMetricsRegistry registry) { + this.registries.add(new RegistryWithSource(source, registry)); + LOG.info("Registering {} with producer.", source); + this.producer.register(source); + } + + @Override + public void stop() { + // Scheduling an event with 0 delay to ensure flushing of metrics one last time before shutdown + this.executor.schedule(this, 0, TimeUnit.SECONDS); + LOG.info("Stopping reporter timer."); + // Allow the scheduled task above to finish, and block for termination (for max 60 seconds) + this.executor.shutdown(); + try { + this.executor.awaitTermination(60, TimeUnit.SECONDS); + } catch (InterruptedException e) { + throw new SamzaException(e); + } + LOG.info("Stopping producer."); + this.producer.stop(); + if (!this.executor.isTerminated()) { + LOG.warn("Unable to shutdown reporter timer."); + } + } + + @Override + public void run() { + try { + innerRun(); + } catch (Exception e) { + // Ignore all exceptions - because subsequent executions of this scheduled task will be suppressed + // by the executor if the current task throws an unhandled exception. + LOG.warn("Error while reporting metrics. Will retry in " + reportingInterval + " seconds.", e); + } + } + + public void innerRun() { + LOG.debug("Begin flushing metrics."); + for (RegistryWithSource registryWithSource : this.registries) { + String source = registryWithSource.getSource(); + ReadableMetricsRegistry registry = registryWithSource.getRegistry(); + LOG.debug("Flushing metrics for {}.", source); + Map<String, Map<String, Object>> metricsMsg = new HashMap<>(); + + // metrics + registry.getGroups().forEach(group -> { + Map<String, Object> groupMsg = new HashMap<>(); + registry.getGroup(group).forEach((name, metric) -> { + if (!shouldIgnore(group, name)) { + metric.visit(new MetricsVisitor() { + @Override + public void counter(Counter counter) { + groupMsg.put(name, counter.getCount()); + } + + @Override + public <T> void gauge(Gauge<T> gauge) { + groupMsg.put(name, gauge.getValue()); + } + + @Override + public void timer(Timer timer) { + groupMsg.put(name, timer.getSnapshot().getAverage()); + } + }); + } + }); + + // dont emit empty groups + if (!groupMsg.isEmpty()) { + metricsMsg.put(group, groupMsg); + } + }); + + // publish to Kafka only if the metricsMsg carries any metrics + if (!metricsMsg.isEmpty()) { + MetricsHeader header = + new MetricsHeader(this.jobName, this.jobId, this.containerName, this.execEnvironmentContainerId, source, + this.version, this.samzaVersion, this.host, this.clock.currentTimeMillis(), this.resetTime); + Metrics metrics = new Metrics(metricsMsg); + LOG.debug("Flushing metrics for {} to {} with header and map: header={}, map={}.", source, out, + header.getAsMap(), metrics.getAsMap()); + MetricsSnapshot metricsSnapshot = new MetricsSnapshot(header, metrics); + Object maybeSerialized = (this.serializer != null) ? this.serializer.toBytes(metricsSnapshot) : metricsSnapshot; + try { + this.producer.send(source, new OutgoingMessageEnvelope(this.out, this.host, null, maybeSerialized)); + // Always flush, since we don't want metrics to get batched up. + this.producer.flush(source); + } catch (Exception e) { + LOG.error(String.format("Exception when flushing metrics for source %s", source), e); + } + } + } + LOG.debug("Finished flushing metrics."); + } + + protected boolean shouldIgnore(String group, String metricName) { + boolean isBlacklisted = this.blacklist.isPresent(); + String fullMetricName = group + "." + metricName; + + if (isBlacklisted && !this.blacklistedMetrics.contains(fullMetricName)) { Review comment: Possible that the same `fullMetricName` exists in different registries? In this case would the second occurrence be filtered out? ########## File path: samza-core/src/main/java/org/apache/samza/metrics/reporter/Metrics.java ########## @@ -0,0 +1,79 @@ +/* + * 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.samza.metrics.reporter; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + + +public class Metrics { Review comment: Is `MetricValues` a better naming? Also contrast with `MetricsHeader` ########## File path: samza-core/src/main/java/org/apache/samza/metrics/reporter/MetricsSnapshotReporterFactory.java ########## @@ -0,0 +1,140 @@ +/* + * 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.samza.metrics.reporter; + +import java.util.Optional; +import org.apache.samza.SamzaException; +import org.apache.samza.config.Config; +import org.apache.samza.config.JobConfig; +import org.apache.samza.config.MetricsConfig; +import org.apache.samza.config.SerializerConfig; +import org.apache.samza.config.StreamConfig; +import org.apache.samza.config.SystemConfig; +import org.apache.samza.metrics.MetricsRegistryMap; +import org.apache.samza.metrics.MetricsReporter; +import org.apache.samza.metrics.MetricsReporterFactory; +import org.apache.samza.serializers.MetricsSnapshotSerdeV2; +import org.apache.samza.serializers.Serde; +import org.apache.samza.serializers.SerdeFactory; +import org.apache.samza.system.SystemFactory; +import org.apache.samza.system.SystemProducer; +import org.apache.samza.system.SystemStream; +import org.apache.samza.util.ReflectionUtil; +import org.apache.samza.util.StreamUtil; +import org.apache.samza.util.SystemClock; +import org.apache.samza.util.Util; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class MetricsSnapshotReporterFactory implements MetricsReporterFactory { + private static final Logger LOG = LoggerFactory.getLogger(MetricsSnapshotReporterFactory.class); + + @Override + public MetricsReporter getMetricsReporter(String reporterName, String containerName, Config config) { + LOG.info("Creating new metrics snapshot reporter."); + MetricsRegistryMap registry = new MetricsRegistryMap(); + + SystemStream systemStream = getSystemStream(reporterName, config); + SystemProducer producer = getProducer(reporterName, config, registry); + int reportingInterval = getReportingInterval(reporterName, config); + String jobName = getJobName(config); + String jobId = getJobId(config); + Serde<MetricsSnapshot> serde = getSerde(reporterName, config); + Optional<String> blacklist = getBlacklist(reporterName, config); + + MetricsSnapshotReporter reporter = + new MetricsSnapshotReporter(producer, systemStream, reportingInterval, jobName, jobId, containerName, + Util.getTaskClassVersion(config), Util.getSamzaVersion(), Util.getLocalHost().getHostName(), serde, + blacklist, SystemClock.instance()); + reporter.register(this.getClass().getSimpleName(), registry); + return reporter; + } + + protected SystemProducer getProducer(String reporterName, Config config, MetricsRegistryMap registry) { + SystemConfig systemConfig = new SystemConfig(config); + String systemName = getSystemStream(reporterName, config).getSystem(); + String systemFactoryClassName = systemConfig.getSystemFactory(systemName) + .orElseThrow(() -> new SamzaException( + String.format("Trying to fetch system factory for system %s, which isn't defined in config.", systemName))); + SystemFactory systemFactory = ReflectionUtil.getObj(systemFactoryClassName, SystemFactory.class); + LOG.info("Got system factory {}.", systemFactory); + SystemProducer producer = systemFactory.getProducer(systemName, config, registry); + LOG.info("Got producer {}.", producer); + return producer; + } + + protected SystemStream getSystemStream(String reporterName, Config config) { + MetricsConfig metricsConfig = new MetricsConfig(config); + String metricsSystemStreamName = metricsConfig.getMetricsSnapshotReporterStream(reporterName) + .orElseThrow(() -> new SamzaException("No metrics stream defined in config.")); + SystemStream systemStream = StreamUtil.getSystemStreamFromNames(metricsSystemStreamName); + LOG.info("Got system stream {}.", systemStream); + return systemStream; + } + + protected Serde<MetricsSnapshot> getSerde(String reporterName, Config config) { + StreamConfig streamConfig = new StreamConfig(config); + SystemConfig systemConfig = new SystemConfig(config); + SystemStream systemStream = getSystemStream(reporterName, config); + + Optional<String> streamSerdeName = streamConfig.getStreamMsgSerde(systemStream); + Optional<String> systemSerdeName = systemConfig.getSystemMsgSerde(systemStream.getSystem()); + String serdeName = streamSerdeName.orElse(systemSerdeName.orElse(null)); + SerializerConfig serializerConfig = new SerializerConfig(config); + Serde<MetricsSnapshot> serde; + if (serdeName != null) { + Optional<String> serdeFactoryClass = serializerConfig.getSerdeFactoryClass(serdeName); + if (serdeFactoryClass.isPresent()) { + SerdeFactory<MetricsSnapshot> serdeFactory = ReflectionUtil.getObj(serdeFactoryClass.get(), SerdeFactory.class); + serde = serdeFactory.getSerde(serdeName, config); + } else { + serde = null; Review comment: Why not `serde = new MetricsSnapshotSerdeV2();`? ########## File path: samza-api/src/main/java/org/apache/samza/metrics/RegistryWithSource.java ########## @@ -16,28 +16,26 @@ * specific language governing permissions and limitations * under the License. */ +package org.apache.samza.metrics; -package org.apache.samza.metrics.reporter - -import java.util.Map -import java.util.HashMap -import scala.beans.BeanProperty +/** + * Simple holder for {@link ReadableMetricsRegistry} and its source, used for implementations of + * {@link MetricsReporter}. + */ +public class RegistryWithSource { Review comment: Why not use ``org.apache.commons.lang3.tuple.Pair`` or use `Map<String, ReadableMetricsRegistry>` for ``registries``? I think the readability is similar, but we introduce 1 less class. -- 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]
