From e66b8022c6cc25a55daac5f5a7661cb1b314564d Mon Sep 17 00:00:00 2001 From: James Petty Date: Tue, 14 Apr 2020 08:09:34 -0400 Subject: [PATCH] Refactor AWS SDK Client Metrics Collection Adds a parent abstract class to PrestoS3FileSystemMetricsCollector so that other SDK clients can share the metrics collector support. Adds reporting for client retry pause time indicating how long the thread was asleep between request retries in the client itself. Fixes the reporting client timings. Previously, when the client retried a request only the first request timings would be recorded in the stats. Now, all request timings are reported individually. --- .../hive/aws/AbstractSdkMetricsCollector.java | 86 +++++++++++++++++++ .../s3/PrestoS3FileSystemMetricCollector.java | 65 +++++++------- .../hive/s3/PrestoS3FileSystemStats.java | 13 +++ 3 files changed, 128 insertions(+), 36 deletions(-) create mode 100644 presto-hive/src/main/java/io/prestosql/plugin/hive/aws/AbstractSdkMetricsCollector.java diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/aws/AbstractSdkMetricsCollector.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/aws/AbstractSdkMetricsCollector.java new file mode 100644 index 00000000000..1188bb6198d --- /dev/null +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/aws/AbstractSdkMetricsCollector.java @@ -0,0 +1,86 @@ +/* + * Licensed 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 io.prestosql.plugin.hive.aws; + +import com.amazonaws.Request; +import com.amazonaws.Response; +import com.amazonaws.metrics.RequestMetricCollector; +import com.amazonaws.util.AWSRequestMetrics; +import com.amazonaws.util.TimingInfo; +import io.airlift.units.Duration; + +import java.util.List; +import java.util.function.Consumer; + +import static com.amazonaws.util.AWSRequestMetrics.Field.ClientExecuteTime; +import static com.amazonaws.util.AWSRequestMetrics.Field.HttpClientRetryCount; +import static com.amazonaws.util.AWSRequestMetrics.Field.HttpRequestTime; +import static com.amazonaws.util.AWSRequestMetrics.Field.RequestCount; +import static com.amazonaws.util.AWSRequestMetrics.Field.RetryPauseTime; +import static com.amazonaws.util.AWSRequestMetrics.Field.ThrottleException; +import static java.util.concurrent.TimeUnit.MILLISECONDS; + +public abstract class AbstractSdkMetricsCollector + extends RequestMetricCollector +{ + @Override + public final void collectMetrics(Request request, Response response) + { + TimingInfo timingInfo = request.getAWSRequestMetrics().getTimingInfo(); + + Number requestCounts = timingInfo.getCounter(RequestCount.name()); + if (requestCounts != null) { + recordRequestCount(requestCounts.longValue()); + } + + Number retryCounts = timingInfo.getCounter(HttpClientRetryCount.name()); + if (retryCounts != null) { + recordRetryCount(retryCounts.longValue()); + } + + Number throttleExceptions = timingInfo.getCounter(ThrottleException.name()); + if (throttleExceptions != null) { + recordThrottleExceptionCount(throttleExceptions.longValue()); + } + + recordSubTimingDurations(timingInfo, HttpRequestTime, this::recordHttpRequestTime); + recordSubTimingDurations(timingInfo, ClientExecuteTime, this::recordClientExecutionTime); + recordSubTimingDurations(timingInfo, RetryPauseTime, this::recordRetryPauseTime); + } + + protected abstract void recordRequestCount(long count); + + protected abstract void recordRetryCount(long count); + + protected abstract void recordThrottleExceptionCount(long count); + + protected abstract void recordHttpRequestTime(Duration duration); + + protected abstract void recordClientExecutionTime(Duration duration); + + protected abstract void recordRetryPauseTime(Duration duration); + + private static void recordSubTimingDurations(TimingInfo timingInfo, AWSRequestMetrics.Field field, Consumer consumer) + { + List subTimings = timingInfo.getAllSubMeasurements(field.name()); + if (subTimings != null) { + for (TimingInfo subTiming : subTimings) { + Double millis = subTiming.getTimeTakenMillisIfKnown(); + if (millis != null) { + consumer.accept(new Duration(millis, MILLISECONDS)); + } + } + } + } +} diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/s3/PrestoS3FileSystemMetricCollector.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/s3/PrestoS3FileSystemMetricCollector.java index 43294a670b4..3c05883c09f 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/s3/PrestoS3FileSystemMetricCollector.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/s3/PrestoS3FileSystemMetricCollector.java @@ -13,23 +13,13 @@ */ package io.prestosql.plugin.hive.s3; -import com.amazonaws.Request; -import com.amazonaws.Response; -import com.amazonaws.metrics.RequestMetricCollector; -import com.amazonaws.util.AWSRequestMetrics; -import com.amazonaws.util.TimingInfo; import io.airlift.units.Duration; +import io.prestosql.plugin.hive.aws.AbstractSdkMetricsCollector; -import static com.amazonaws.util.AWSRequestMetrics.Field.ClientExecuteTime; -import static com.amazonaws.util.AWSRequestMetrics.Field.HttpClientRetryCount; -import static com.amazonaws.util.AWSRequestMetrics.Field.HttpRequestTime; -import static com.amazonaws.util.AWSRequestMetrics.Field.RequestCount; -import static com.amazonaws.util.AWSRequestMetrics.Field.ThrottleException; import static java.util.Objects.requireNonNull; -import static java.util.concurrent.TimeUnit.MILLISECONDS; public class PrestoS3FileSystemMetricCollector - extends RequestMetricCollector + extends AbstractSdkMetricsCollector { private final PrestoS3FileSystemStats stats; @@ -39,35 +29,38 @@ public PrestoS3FileSystemMetricCollector(PrestoS3FileSystemStats stats) } @Override - public void collectMetrics(Request request, Response response) + protected void recordRequestCount(long count) { - AWSRequestMetrics metrics = request.getAWSRequestMetrics(); - - TimingInfo timingInfo = metrics.getTimingInfo(); - Number requestCounts = timingInfo.getCounter(RequestCount.name()); - Number retryCounts = timingInfo.getCounter(HttpClientRetryCount.name()); - Number throttleExceptions = timingInfo.getCounter(ThrottleException.name()); - TimingInfo requestTime = timingInfo.getSubMeasurement(HttpRequestTime.name()); - TimingInfo clientExecuteTime = timingInfo.getSubMeasurement(ClientExecuteTime.name()); + stats.updateAwsRequestCount(count); + } - if (requestCounts != null) { - stats.updateAwsRequestCount(requestCounts.longValue()); - } + @Override + protected void recordRetryCount(long count) + { + stats.updateAwsRetryCount(count); + } - if (retryCounts != null) { - stats.updateAwsRetryCount(retryCounts.longValue()); - } + @Override + protected void recordThrottleExceptionCount(long count) + { + stats.updateAwsThrottleExceptionsCount(count); + } - if (throttleExceptions != null) { - stats.updateAwsThrottleExceptionsCount(throttleExceptions.longValue()); - } + @Override + protected void recordHttpRequestTime(Duration duration) + { + stats.addAwsRequestTime(duration); + } - if (requestTime != null && requestTime.getTimeTakenMillisIfKnown() != null) { - stats.addAwsRequestTime(new Duration(requestTime.getTimeTakenMillisIfKnown(), MILLISECONDS)); - } + @Override + protected void recordClientExecutionTime(Duration duration) + { + stats.addAwsClientExecuteTime(duration); + } - if (clientExecuteTime != null && clientExecuteTime.getTimeTakenMillisIfKnown() != null) { - stats.addAwsClientExecuteTime(new Duration(clientExecuteTime.getTimeTakenMillisIfKnown(), MILLISECONDS)); - } + @Override + protected void recordRetryPauseTime(Duration duration) + { + stats.addAwsClientRetryPauseTime(duration); } } diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/s3/PrestoS3FileSystemStats.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/s3/PrestoS3FileSystemStats.java index e82825f48f3..16886ed6222 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/s3/PrestoS3FileSystemStats.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/s3/PrestoS3FileSystemStats.java @@ -51,6 +51,7 @@ public class PrestoS3FileSystemStats private final CounterStat awsThrottleExceptions = new CounterStat(); private final TimeStat awsRequestTime = new TimeStat(MILLISECONDS); private final TimeStat awsClientExecuteTime = new TimeStat(MILLISECONDS); + private final TimeStat awsClientRetryPauseTime = new TimeStat(MILLISECONDS); @Managed @Nested @@ -185,6 +186,13 @@ public TimeStat getAwsClientExecuteTime() return awsClientExecuteTime; } + @Managed + @Nested + public TimeStat getAwsClientRetryPauseTime() + { + return awsClientRetryPauseTime; + } + @Managed @Nested public CounterStat getGetObjectRetries() @@ -302,6 +310,11 @@ public void addAwsClientExecuteTime(Duration duration) awsClientExecuteTime.add(duration); } + public void addAwsClientRetryPauseTime(Duration duration) + { + awsClientRetryPauseTime.add(duration); + } + public void newGetObjectRetry() { getObjectRetries.update(1);