1
0

[HUDI-2801] Add Amazon CloudWatch metrics reporter (#4081)

This commit is contained in:
Udit Mehrotra
2021-11-25 13:33:16 -08:00
committed by GitHub
parent 8e1379384a
commit e0125a7911
15 changed files with 799 additions and 13 deletions

View File

@@ -0,0 +1,315 @@
/*
* 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.aws.cloudwatch;
import org.apache.hudi.aws.credentials.HoodieAWSCredentialsProviderFactory;
import org.apache.hudi.common.util.Option;
import com.amazonaws.services.cloudwatch.AmazonCloudWatchAsync;
import com.amazonaws.services.cloudwatch.AmazonCloudWatchAsyncClientBuilder;
import com.amazonaws.services.cloudwatch.model.Dimension;
import com.amazonaws.services.cloudwatch.model.MetricDatum;
import com.amazonaws.services.cloudwatch.model.PutMetricDataRequest;
import com.amazonaws.services.cloudwatch.model.PutMetricDataResult;
import com.amazonaws.services.cloudwatch.model.StandardUnit;
import com.codahale.metrics.Clock;
import com.codahale.metrics.Counter;
import com.codahale.metrics.Counting;
import com.codahale.metrics.Gauge;
import com.codahale.metrics.Histogram;
import com.codahale.metrics.Meter;
import com.codahale.metrics.MetricFilter;
import com.codahale.metrics.MetricRegistry;
import com.codahale.metrics.ScheduledReporter;
import com.codahale.metrics.Timer;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.util.ArrayList;
import java.util.Date;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.SortedMap;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
/**
* A reporter for publishing metrics to Amazon CloudWatch. It is responsible for collecting, converting DropWizard
* metrics to CloudWatch metrics and composing metrics payload.
*/
public class CloudWatchReporter extends ScheduledReporter {
static final String DIMENSION_TABLE_NAME_KEY = "Table";
static final String DIMENSION_METRIC_TYPE_KEY = "Metric Type";
static final String DIMENSION_GAUGE_TYPE_VALUE = "gauge";
static final String DIMENSION_COUNT_TYPE_VALUE = "count";
private static final Logger LOG = LogManager.getLogger(CloudWatchReporter.class);
private final AmazonCloudWatchAsync cloudWatchClientAsync;
private final Clock clock;
private final String prefix;
private final String namespace;
private final int maxDatumsPerRequest;
public static Builder forRegistry(MetricRegistry registry) {
return new Builder(registry);
}
public static class Builder {
private MetricRegistry registry;
private Clock clock;
private String prefix;
private TimeUnit rateUnit;
private TimeUnit durationUnit;
private MetricFilter filter;
private String namespace;
private int maxDatumsPerRequest;
private Builder(MetricRegistry registry) {
this.registry = registry;
this.clock = Clock.defaultClock();
this.rateUnit = TimeUnit.SECONDS;
this.durationUnit = TimeUnit.MILLISECONDS;
this.filter = MetricFilter.ALL;
this.maxDatumsPerRequest = 20;
}
public Builder withClock(Clock clock) {
this.clock = clock;
return this;
}
public Builder prefixedWith(String prefix) {
this.prefix = prefix;
return this;
}
public Builder convertRatesTo(TimeUnit rateUnit) {
this.rateUnit = rateUnit;
return this;
}
public Builder convertDurationsTo(TimeUnit durationUnit) {
this.durationUnit = durationUnit;
return this;
}
public Builder filter(MetricFilter filter) {
this.filter = filter;
return this;
}
public Builder namespace(String namespace) {
this.namespace = namespace;
return this;
}
public Builder maxDatumsPerRequest(int maxDatumsPerRequest) {
this.maxDatumsPerRequest = maxDatumsPerRequest;
return this;
}
public CloudWatchReporter build(Properties props) {
return new CloudWatchReporter(registry,
getAmazonCloudWatchClient(props),
clock,
prefix,
namespace,
maxDatumsPerRequest,
filter,
rateUnit,
durationUnit);
}
CloudWatchReporter build(AmazonCloudWatchAsync amazonCloudWatchAsync) {
return new CloudWatchReporter(registry,
amazonCloudWatchAsync,
clock,
prefix,
namespace,
maxDatumsPerRequest,
filter,
rateUnit,
durationUnit);
}
}
protected CloudWatchReporter(MetricRegistry registry,
AmazonCloudWatchAsync cloudWatchClientAsync,
Clock clock,
String prefix,
String namespace,
int maxDatumsPerRequest,
MetricFilter filter,
TimeUnit rateUnit,
TimeUnit durationUnit) {
super(registry, "hudi-cloudWatch-reporter", filter, rateUnit, durationUnit);
this.cloudWatchClientAsync = cloudWatchClientAsync;
this.clock = clock;
this.prefix = prefix;
this.namespace = namespace;
this.maxDatumsPerRequest = maxDatumsPerRequest;
}
private static AmazonCloudWatchAsync getAmazonCloudWatchClient(Properties props) {
return AmazonCloudWatchAsyncClientBuilder.standard()
.withCredentials(HoodieAWSCredentialsProviderFactory.getAwsCredentialsProvider(props))
.build();
}
@Override
public void report(SortedMap<String, Gauge> gauges,
SortedMap<String, Counter> counters,
SortedMap<String, Histogram> histograms,
SortedMap<String, Meter> meters,
SortedMap<String, Timer> timers) {
LOG.info("Reporting Metrics to CloudWatch.");
final long timestampMilliSec = clock.getTime();
List<MetricDatum> metricsData = new ArrayList<>();
for (Map.Entry<String, Gauge> entry : gauges.entrySet()) {
processGauge(entry.getKey(), entry.getValue(), timestampMilliSec, metricsData);
}
for (Map.Entry<String, Counter> entry : counters.entrySet()) {
processCounter(entry.getKey(), entry.getValue(), timestampMilliSec, metricsData);
}
for (Map.Entry<String, Histogram> entry : histograms.entrySet()) {
processCounter(entry.getKey(), entry.getValue(), timestampMilliSec, metricsData);
//TODO: Publish other Histogram metrics to cloud watch
}
for (Map.Entry<String, Meter> entry : meters.entrySet()) {
processCounter(entry.getKey(), entry.getValue(), timestampMilliSec, metricsData);
//TODO: Publish other Meter metrics to cloud watch
}
for (Map.Entry<String, Timer> entry : timers.entrySet()) {
processCounter(entry.getKey(), entry.getValue(), timestampMilliSec, metricsData);
//TODO: Publish other Timer metrics to cloud watch
}
report(metricsData);
}
private void report(List<MetricDatum> metricsData) {
List<Future<PutMetricDataResult>> cloudWatchFutures = new ArrayList<>(metricsData.size());
List<List<MetricDatum>> partitions = new ArrayList<>();
for (int i = 0; i < metricsData.size(); i += maxDatumsPerRequest) {
int end = Math.min(metricsData.size(), i + maxDatumsPerRequest);
partitions.add(metricsData.subList(i, end));
}
for (List<MetricDatum> partition : partitions) {
PutMetricDataRequest request = new PutMetricDataRequest()
.withNamespace(namespace)
.withMetricData(partition);
cloudWatchFutures.add(cloudWatchClientAsync.putMetricDataAsync(request));
}
for (final Future<PutMetricDataResult> cloudWatchFuture : cloudWatchFutures) {
try {
cloudWatchFuture.get(30, TimeUnit.SECONDS);
} catch (final Exception ex) {
LOG.error("Error reporting metrics to CloudWatch. The data in this CloudWatch request "
+ "may have been discarded, and not made it to CloudWatch.", ex);
}
}
}
private void processGauge(final String metricName,
final Gauge<?> gauge,
final long timestampMilliSec,
final List<MetricDatum> metricData) {
Option.ofNullable(gauge.getValue())
.toJavaOptional()
.filter(value -> value instanceof Number)
.map(value -> (Number) value)
.ifPresent(value -> stageMetricDatum(metricName,
value.doubleValue(),
DIMENSION_GAUGE_TYPE_VALUE,
StandardUnit.None,
timestampMilliSec,
metricData));
}
private void processCounter(final String metricName,
final Counting counter,
final long timestampMilliSec,
final List<MetricDatum> metricData) {
stageMetricDatum(metricName,
counter.getCount(),
DIMENSION_COUNT_TYPE_VALUE,
StandardUnit.Count,
timestampMilliSec,
metricData);
}
private void stageMetricDatum(String metricName,
double metricValue,
String metricType,
StandardUnit standardUnit,
long timestampMilliSec,
List<MetricDatum> metricData) {
String[] metricNameParts = metricName.split("\\.", 2);
String tableName = metricNameParts[0];
metricData.add(new MetricDatum()
.withTimestamp(new Date(timestampMilliSec))
.withMetricName(prefix(metricNameParts[1]))
.withValue(metricValue)
.withDimensions(getDimensions(tableName, metricType))
.withUnit(standardUnit));
}
private List<Dimension> getDimensions(String tableName, String metricType) {
List<Dimension> dimensions = new ArrayList<>();
dimensions.add(new Dimension()
.withName(DIMENSION_TABLE_NAME_KEY)
.withValue(tableName));
dimensions.add(new Dimension()
.withName(DIMENSION_METRIC_TYPE_KEY)
.withValue(metricType));
return dimensions;
}
private String prefix(String... components) {
return MetricRegistry.name(prefix, components);
}
@Override
public void stop() {
try {
super.stop();
} finally {
try {
cloudWatchClientAsync.shutdown();
} catch (Exception ex) {
LOG.warn("Exception while shutting down CloudWatch client.", ex);
}
}
}
}

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.config;
import org.apache.hudi.common.config.ConfigProperty;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.util.Option;
import com.amazonaws.regions.RegionUtils;
@@ -34,14 +35,16 @@ public class AWSLockConfiguration {
public static final ConfigProperty<String> DYNAMODB_LOCK_TABLE_NAME = ConfigProperty
.key(DYNAMODB_BASED_LOCK_PROPERTY_PREFIX + "table")
.noDefaultValue()
.sinceVersion("0.10.0")
.withDocumentation("For DynamoDB based lock provider, the name of the DynamoDB table acting as lock table");
public static final ConfigProperty<String> DYNAMODB_LOCK_PARTITION_KEY = ConfigProperty
.key(DYNAMODB_BASED_LOCK_PROPERTY_PREFIX + "partition_key")
.noDefaultValue()
.sinceVersion("0.10.0")
.withInferFunction(cfg -> {
if (cfg.contains(HoodieWriteConfig.TBL_NAME)) {
return Option.of(cfg.getString(HoodieWriteConfig.TBL_NAME));
if (cfg.contains(HoodieTableConfig.NAME)) {
return Option.of(cfg.getString(HoodieTableConfig.NAME));
}
return Option.empty();
})
@@ -52,6 +55,7 @@ public class AWSLockConfiguration {
public static final ConfigProperty<String> DYNAMODB_LOCK_REGION = ConfigProperty
.key(DYNAMODB_BASED_LOCK_PROPERTY_PREFIX + "region")
.defaultValue("us-east-1")
.sinceVersion("0.10.0")
.withInferFunction(cfg -> {
String regionFromEnv = System.getenv("AWS_REGION");
if (regionFromEnv != null) {
@@ -65,20 +69,24 @@ public class AWSLockConfiguration {
public static final ConfigProperty<String> DYNAMODB_LOCK_BILLING_MODE = ConfigProperty
.key(DYNAMODB_BASED_LOCK_PROPERTY_PREFIX + "billing_mode")
.defaultValue(BillingMode.PAY_PER_REQUEST.name())
.sinceVersion("0.10.0")
.withDocumentation("For DynamoDB based lock provider, by default it is PAY_PER_REQUEST mode");
public static final ConfigProperty<String> DYNAMODB_LOCK_READ_CAPACITY = ConfigProperty
.key(DYNAMODB_BASED_LOCK_PROPERTY_PREFIX + "read_capacity")
.defaultValue("20")
.sinceVersion("0.10.0")
.withDocumentation("For DynamoDB based lock provider, read capacity units when using PROVISIONED billing mode");
public static final ConfigProperty<String> DYNAMODB_LOCK_WRITE_CAPACITY = ConfigProperty
.key(DYNAMODB_BASED_LOCK_PROPERTY_PREFIX + "write_capacity")
.defaultValue("10")
.sinceVersion("0.10.0")
.withDocumentation("For DynamoDB based lock provider, write capacity units when using PROVISIONED billing mode");
public static final ConfigProperty<String> DYNAMODB_LOCK_TABLE_CREATION_TIMEOUT = ConfigProperty
.key(DYNAMODB_BASED_LOCK_PROPERTY_PREFIX + "table_creation_timeout")
.defaultValue(String.valueOf(10 * 60 * 1000))
.sinceVersion("0.10.0")
.withDocumentation("For DynamoDB based lock provider, the maximum number of milliseconds to wait for creating DynamoDB table");
}

View File

@@ -40,23 +40,27 @@ import static org.apache.hudi.config.AWSLockConfiguration.DYNAMODB_LOCK_WRITE_CA
* Configurations used by the AWS credentials and AWS DynamoDB based lock.
*/
@Immutable
@ConfigClassProperty(name = "AWS credential Configs",
groupName = ConfigGroups.Names.AWS_DYNAMO_DB,
description = "Configurations used for AWS credentials to get AWS resources.")
@ConfigClassProperty(name = "Amazon Web Services Configs",
groupName = ConfigGroups.Names.AWS,
description = "Amazon Web Services configurations to access resources like Amazon DynamoDB (for locks),"
+ " Amazon CloudWatch (metrics).")
public class HoodieAWSConfig extends HoodieConfig {
public static final ConfigProperty<String> AWS_ACCESS_KEY = ConfigProperty
.key("hoodie.aws.access.key")
.noDefaultValue()
.sinceVersion("0.10.0")
.withDocumentation("AWS access key id");
public static final ConfigProperty<String> AWS_SECRET_KEY = ConfigProperty
.key("hoodie.aws.secret.key")
.noDefaultValue()
.sinceVersion("0.10.0")
.withDocumentation("AWS secret key");
public static final ConfigProperty<String> AWS_SESSION_TOKEN = ConfigProperty
.key("hoodie.aws.session.token")
.noDefaultValue()
.sinceVersion("0.10.0")
.withDocumentation("AWS session token");
private HoodieAWSConfig() {

View File

@@ -0,0 +1,87 @@
/*
* 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.config;
import org.apache.hudi.common.config.ConfigClassProperty;
import org.apache.hudi.common.config.ConfigGroups;
import org.apache.hudi.common.config.ConfigProperty;
import org.apache.hudi.common.config.HoodieConfig;
import java.util.Properties;
@ConfigClassProperty(
name = "Metrics Configurations for Amazon CloudWatch",
groupName = ConfigGroups.Names.METRICS,
description =
"Enables reporting on Hudi metrics using Amazon CloudWatch. "
+ " Hudi publishes metrics on every commit, clean, rollback etc.")
public class HoodieMetricsCloudWatchConfig extends HoodieConfig {
public static final String CLOUDWATCH_PREFIX = "hoodie.metrics.cloudwatch";
public static final ConfigProperty<Integer> REPORT_PERIOD_SECONDS = ConfigProperty
.key(CLOUDWATCH_PREFIX + ".report.period.seconds")
.defaultValue(60)
.sinceVersion("0.10.0")
.withDocumentation("Reporting interval in seconds");
public static final ConfigProperty<String> METRIC_PREFIX = ConfigProperty
.key(CLOUDWATCH_PREFIX + ".metric.prefix")
.defaultValue("")
.sinceVersion("0.10.0")
.withDocumentation("Metric prefix of reporter");
public static final ConfigProperty<String> METRIC_NAMESPACE = ConfigProperty
.key(CLOUDWATCH_PREFIX + ".namespace")
.defaultValue("Hudi")
.sinceVersion("0.10.0")
.withDocumentation("Namespace of reporter");
/*
Amazon CloudWatch allows a maximum of 20 metrics per request. Choosing this as the default maximum.
Reference: https://docs.aws.amazon.com/AmazonCloudWatch/latest/APIReference/API_PutMetricData.html
*/
public static final ConfigProperty<Integer> MAX_DATUMS_PER_REQUEST =
ConfigProperty.key(CLOUDWATCH_PREFIX + ".maxDatumsPerRequest")
.defaultValue(20)
.sinceVersion("0.10.0")
.withDocumentation("Max number of Datums per request");
public HoodieMetricsCloudWatchConfig() {
super();
}
public static HoodieMetricsCloudWatchConfig.Builder newBuilder() {
return new HoodieMetricsCloudWatchConfig.Builder();
}
public static class Builder {
private HoodieMetricsCloudWatchConfig hoodieMetricsCloudWatchConfig = new HoodieMetricsCloudWatchConfig();
public HoodieMetricsCloudWatchConfig.Builder fromProperties(Properties props) {
this.hoodieMetricsCloudWatchConfig.getProps().putAll(props);
return this;
}
public HoodieMetricsCloudWatchConfig build() {
hoodieMetricsCloudWatchConfig.setDefaults(HoodieMetricsCloudWatchConfig.class.getName());
return hoodieMetricsCloudWatchConfig;
}
}
}

View File

@@ -0,0 +1,183 @@
/*
* 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.aws.cloudwatch;
import com.amazonaws.services.cloudwatch.AmazonCloudWatchAsync;
import com.amazonaws.services.cloudwatch.model.Dimension;
import com.amazonaws.services.cloudwatch.model.MetricDatum;
import com.amazonaws.services.cloudwatch.model.PutMetricDataRequest;
import com.amazonaws.services.cloudwatch.model.PutMetricDataResult;
import com.codahale.metrics.Clock;
import com.codahale.metrics.Counter;
import com.codahale.metrics.ExponentiallyDecayingReservoir;
import com.codahale.metrics.Gauge;
import com.codahale.metrics.Histogram;
import com.codahale.metrics.Meter;
import com.codahale.metrics.MetricFilter;
import com.codahale.metrics.MetricRegistry;
import com.codahale.metrics.Timer;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.ArgumentCaptor;
import org.mockito.ArgumentMatchers;
import org.mockito.Captor;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.jupiter.MockitoExtension;
import java.util.List;
import java.util.SortedMap;
import java.util.TreeMap;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import static org.apache.hudi.aws.cloudwatch.CloudWatchReporter.DIMENSION_COUNT_TYPE_VALUE;
import static org.apache.hudi.aws.cloudwatch.CloudWatchReporter.DIMENSION_GAUGE_TYPE_VALUE;
import static org.apache.hudi.aws.cloudwatch.CloudWatchReporter.DIMENSION_METRIC_TYPE_KEY;
import static org.apache.hudi.aws.cloudwatch.CloudWatchReporter.DIMENSION_TABLE_NAME_KEY;
import static org.junit.jupiter.api.Assertions.assertEquals;
@ExtendWith(MockitoExtension.class)
public class TestCloudWatchReporter {
private static final String NAMESPACE = "Hudi Test";
private static final String PREFIX = "testPrefix";
private static final String TABLE_NAME = "testTable";
private static final int MAX_DATUMS_PER_REQUEST = 2;
@Mock
MetricRegistry metricRegistry;
@Mock
AmazonCloudWatchAsync cloudWatchAsync;
@Mock
CompletableFuture<PutMetricDataResult> cloudWatchFuture;
@Captor
ArgumentCaptor<PutMetricDataRequest> putMetricDataRequestCaptor;
CloudWatchReporter reporter;
@BeforeEach
public void setup() {
reporter = CloudWatchReporter.forRegistry(metricRegistry)
.namespace(NAMESPACE)
.prefixedWith(PREFIX)
.maxDatumsPerRequest(MAX_DATUMS_PER_REQUEST)
.withClock(Clock.defaultClock())
.filter(MetricFilter.ALL)
.convertRatesTo(TimeUnit.SECONDS)
.convertDurationsTo(TimeUnit.MILLISECONDS)
.build(cloudWatchAsync);
Mockito.when(cloudWatchAsync.putMetricDataAsync(ArgumentMatchers.any())).thenReturn(cloudWatchFuture);
}
@Test
public void testReporter() {
SortedMap<String, Gauge> gauges = new TreeMap<>();
Gauge<Long> gauge1 = () -> 100L;
Gauge<Double> gauge2 = () -> 100.1;
gauges.put(TABLE_NAME + ".gauge1", gauge1);
gauges.put(TABLE_NAME + ".gauge2", gauge2);
SortedMap<String, Counter> counters = new TreeMap<>();
Counter counter1 = new Counter();
counter1.inc(200);
counters.put(TABLE_NAME + ".counter1", counter1);
SortedMap<String, Histogram> histograms = new TreeMap<>();
Histogram histogram1 = new Histogram(new ExponentiallyDecayingReservoir());
histogram1.update(300);
histograms.put(TABLE_NAME + ".histogram1", histogram1);
SortedMap<String, Meter> meters = new TreeMap<>();
Meter meter1 = new Meter();
meter1.mark(400);
meters.put(TABLE_NAME + ".meter1", meter1);
SortedMap<String, Timer> timers = new TreeMap<>();
Timer timer1 = new Timer();
timer1.update(100, TimeUnit.SECONDS);
timers.put(TABLE_NAME + ".timer1", timer1);
Mockito.when(metricRegistry.getGauges(MetricFilter.ALL)).thenReturn(gauges);
Mockito.when(metricRegistry.getCounters(MetricFilter.ALL)).thenReturn(counters);
Mockito.when(metricRegistry.getHistograms(MetricFilter.ALL)).thenReturn(histograms);
Mockito.when(metricRegistry.getMeters(MetricFilter.ALL)).thenReturn(meters);
Mockito.when(metricRegistry.getTimers(MetricFilter.ALL)).thenReturn(timers);
reporter.report();
// Since there are 6 metrics in total, and max datums per request is 2 we would expect 3 calls to CloudWatch
// with 2 datums in each
Mockito.verify(cloudWatchAsync, Mockito.times(3)).putMetricDataAsync(putMetricDataRequestCaptor.capture());
Assertions.assertEquals(NAMESPACE, putMetricDataRequestCaptor.getValue().getNamespace());
List<PutMetricDataRequest> putMetricDataRequests = putMetricDataRequestCaptor.getAllValues();
putMetricDataRequests.forEach(request -> assertEquals(2, request.getMetricData().size()));
List<MetricDatum> metricDataBatch1 = putMetricDataRequests.get(0).getMetricData();
assertEquals(PREFIX + ".gauge1", metricDataBatch1.get(0).getMetricName());
assertEquals(Double.valueOf(gauge1.getValue()), metricDataBatch1.get(0).getValue());
assertDimensions(metricDataBatch1.get(0).getDimensions(), DIMENSION_GAUGE_TYPE_VALUE);
assertEquals(PREFIX + ".gauge2", metricDataBatch1.get(1).getMetricName());
assertEquals(gauge2.getValue(), metricDataBatch1.get(1).getValue());
assertDimensions(metricDataBatch1.get(1).getDimensions(), DIMENSION_GAUGE_TYPE_VALUE);
List<MetricDatum> metricDataBatch2 = putMetricDataRequests.get(1).getMetricData();
assertEquals(PREFIX + ".counter1", metricDataBatch2.get(0).getMetricName());
assertEquals(counter1.getCount(), metricDataBatch2.get(0).getValue().longValue());
assertDimensions(metricDataBatch2.get(0).getDimensions(), DIMENSION_COUNT_TYPE_VALUE);
assertEquals(PREFIX + ".histogram1", metricDataBatch2.get(1).getMetricName());
assertEquals(histogram1.getCount(), metricDataBatch2.get(1).getValue().longValue());
assertDimensions(metricDataBatch2.get(1).getDimensions(), DIMENSION_COUNT_TYPE_VALUE);
List<MetricDatum> metricDataBatch3 = putMetricDataRequests.get(2).getMetricData();
assertEquals(PREFIX + ".meter1", metricDataBatch3.get(0).getMetricName());
assertEquals(meter1.getCount(), metricDataBatch3.get(0).getValue().longValue());
assertDimensions(metricDataBatch3.get(0).getDimensions(), DIMENSION_COUNT_TYPE_VALUE);
assertEquals(PREFIX + ".timer1", metricDataBatch3.get(1).getMetricName());
assertEquals(timer1.getCount(), metricDataBatch3.get(1).getValue().longValue());
assertDimensions(metricDataBatch3.get(1).getDimensions(), DIMENSION_COUNT_TYPE_VALUE);
reporter.stop();
Mockito.verify(cloudWatchAsync).shutdown();
}
private void assertDimensions(List<Dimension> actualDimensions, String metricTypeDimensionVal) {
assertEquals(2, actualDimensions.size());
Dimension expectedTableNameDimension = new Dimension()
.withName(DIMENSION_TABLE_NAME_KEY)
.withValue(TABLE_NAME);
Dimension expectedMetricTypeDimension = new Dimension()
.withName(DIMENSION_METRIC_TYPE_KEY)
.withValue(metricTypeDimensionVal);
assertEquals(expectedTableNameDimension, actualDimensions.get(0));
assertEquals(expectedMetricTypeDimension, actualDimensions.get(1));
}
}