diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 080240d9587c4..e9dd94ff4ed88 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -323,18 +323,42 @@ public class AbfsConfiguration{ DefaultValue = DEFAULT_METRIC_ANALYSIS_TIMEOUT_MS) private int metricAnalysisTimeout; - @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_METRIC_URI, - DefaultValue = EMPTY_STRING) - private String metricUri; - - @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_METRIC_ACCOUNT_NAME, + @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_METRICS_ACCOUNT_NAME, DefaultValue = EMPTY_STRING) private String metricAccount; - @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_METRIC_ACCOUNT_KEY, + @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_METRICS_ACCOUNT_KEY, DefaultValue = EMPTY_STRING) private String metricAccountKey; + @BooleanConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_METRICS_COLLECTION_ENABLED, + DefaultValue = DEFAULT_METRICS_COLLECTION_ENABLED) + private boolean metricsCollectionEnabled; + + @BooleanConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_METRICS_SHOULD_EMIT_ON_IDLE_TIME, + DefaultValue = DEFAULT_METRICS_SHOULD_EMIT_ON_IDLE_TIME) + private boolean shouldEmitMetricsOnIdleTime; + + @LongConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_METRICS_EMIT_THRESHOLD, + DefaultValue = DEFAULT_METRICS_EMIT_THRESHOLD) + private long metricsEmitThreshold; + + @LongConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_METRICS_EMIT_THRESHOLD_INTERVAL_SECS, + DefaultValue = DEFAULT_METRICS_EMIT_THRESHOLD_INTERVAL_SECS) + private long metricsEmitThresholdIntervalInSecs; + + @LongConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_METRICS_EMIT_INTERVAL_MINS, + DefaultValue = DEFAULT_METRICS_EMIT_INTERVAL_MINS) + private long metricsEmitIntervalInMins; + + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_METRICS_MAX_CALLS_PER_SECOND, + DefaultValue = DEFAULT_METRICS_MAX_CALLS_PER_SECOND) + private int maxMetricsCallsPerSecond; + + @BooleanConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_METRICS_BACKOFF_RETRY_ENABLED, + DefaultValue = DEFAULT_METRICS_BACKOFF_RETRY_ENABLED) + private boolean backoffRetryMetricsEnabled; + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ACCOUNT_OPERATION_IDLE_TIMEOUT, DefaultValue = DEFAULT_ACCOUNT_OPERATION_IDLE_TIMEOUT_MS) private int accountOperationIdleTimeout; @@ -1294,10 +1318,6 @@ public int getMetricAnalysisTimeout() { return this.metricAnalysisTimeout; } - public String getMetricUri() { - return metricUri; - } - public String getMetricAccount() { return metricAccount; } @@ -1306,6 +1326,34 @@ public String getMetricAccountKey() { return metricAccountKey; } + public boolean isMetricsCollectionEnabled() { + return metricsCollectionEnabled; + } + + public boolean shouldEmitMetricsOnIdleTime() { + return shouldEmitMetricsOnIdleTime; + } + + public long getMetricsEmitThreshold() { + return metricsEmitThreshold; + } + + public long getMetricsEmitIntervalInMins() { + return metricsEmitIntervalInMins; + } + + public long getMetricsEmitThresholdIntervalInSecs() { + return metricsEmitThresholdIntervalInSecs; + } + + public int getMaxMetricsCallsPerSecond() { + return maxMetricsCallsPerSecond; + } + + public boolean isBackoffRetryMetricsEnabled() { + return backoffRetryMetricsEnabled; + } + public int getAccountOperationIdleTimeout() { return accountOperationIdleTimeout; } @@ -1394,7 +1442,7 @@ public TracingHeaderFormat getTracingHeaderFormat() { } public MetricFormat getMetricFormat() { - return getEnum(FS_AZURE_METRIC_FORMAT, MetricFormat.EMPTY); + return getEnum(FS_AZURE_METRICS_FORMAT, MetricFormat.INTERNAL_METRIC_FORMAT); } public AuthType getAuthType(String accountName) { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsCountersImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsCountersImpl.java index 8bc7d5017bc43..363ed34025a7f 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsCountersImpl.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsCountersImpl.java @@ -73,6 +73,7 @@ import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.SEND_REQUESTS; import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.SERVER_UNAVAILABLE; import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.WRITE_THROTTLES; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; import static org.apache.hadoop.fs.azurebfs.enums.AbfsBackoffMetricsEnum.TOTAL_NUMBER_OF_REQUESTS; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore; import static org.apache.hadoop.util.Time.now; @@ -201,23 +202,39 @@ public void initializeWriteResourceUtilizationMetrics() { @Override - public void initializeMetrics(MetricFormat metricFormat) { + public void initializeMetrics(final MetricFormat metricFormat, + final AbfsConfiguration abfsConfiguration) { switch (metricFormat) { - case INTERNAL_BACKOFF_METRIC_FORMAT: - abfsBackoffMetrics = new AbfsBackoffMetrics(); - break; - case INTERNAL_FOOTER_METRIC_FORMAT: - abfsReadFooterMetrics = new AbfsReadFooterMetrics(); - break; - case INTERNAL_METRIC_FORMAT: - abfsBackoffMetrics = new AbfsBackoffMetrics(); - abfsReadFooterMetrics = new AbfsReadFooterMetrics(); - break; - default: - break; + case INTERNAL_BACKOFF_METRIC_FORMAT: + abfsBackoffMetrics = new AbfsBackoffMetrics( + abfsConfiguration.isBackoffRetryMetricsEnabled()); + break; + case INTERNAL_FOOTER_METRIC_FORMAT: + initializeReadFooterMetrics(); + break; + case INTERNAL_METRIC_FORMAT: + abfsBackoffMetrics = new AbfsBackoffMetrics( + abfsConfiguration.isBackoffRetryMetricsEnabled()); + initializeReadFooterMetrics(); + break; + default: + break; } } + /** + * Initialize the read footer metrics. + * In case the metrics are already initialized, + * create a new instance with the existing map. + */ + private void initializeReadFooterMetrics() { + abfsReadFooterMetrics = new AbfsReadFooterMetrics( + abfsReadFooterMetrics == null + ? null + : abfsReadFooterMetrics.getFileTypeMetricsMap() + ); + } + /** * Look up a Metric from registered set. * @@ -373,10 +390,9 @@ public DurationTracker trackDuration(String key) { @Override public String toString() { - String metric = ""; + String metric = EMPTY_STRING; if (abfsBackoffMetrics != null) { - long totalNoRequests = getAbfsBackoffMetrics().getMetricValue(TOTAL_NUMBER_OF_REQUESTS); - if (totalNoRequests > 0) { + if (getAbfsBackoffMetrics().getMetricValue(TOTAL_NUMBER_OF_REQUESTS) > 0) { metric += "#BO:" + getAbfsBackoffMetrics().toString(); } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index e41f73af77635..9ed20251043d2 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -216,6 +216,7 @@ public void initialize(URI uri, Configuration configuration) .withBlockFactory(blockFactory) .withBlockOutputActiveBlocks(blockOutputActiveBlocks) .withBackReference(new BackReference(this)) + .withFileSystemId(this.fileSystemId) .build(); this.abfsStore = new AzureBlobFileSystemStore(systemStoreBuilder); @@ -841,18 +842,6 @@ public synchronized void close() throws IOException { if (isClosed()) { return; } - if (getAbfsStore().getClient().isMetricCollectionEnabled()) { - TracingContext tracingMetricContext = new TracingContext( - clientCorrelationId, - fileSystemId, FSOperationType.GET_ATTR, true, - tracingHeaderFormat, - listener, abfsCounters.toString()); - try { - getAbfsClient().getMetricCall(tracingMetricContext); - } catch (IOException e) { - throw new IOException(e); - } - } // does all the delete-on-exit calls, and may be slow. super.close(); LOG.debug("AzureBlobFileSystem.close"); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 93f1cfb436c9e..de4bc79d55aa8 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -193,6 +193,7 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport { private final IdentityTransformerInterface identityTransformer; private final AbfsPerfTracker abfsPerfTracker; private final AbfsCounters abfsCounters; + private final String fileSystemId; /** * The set of directories where we should store files as append blobs. @@ -258,6 +259,7 @@ public AzureBlobFileSystemStore( boolean useHttps = (usingOauth || abfsConfiguration.isHttpsAlwaysUsed()) ? true : abfsStoreBuilder.isSecureScheme; this.abfsPerfTracker = new AbfsPerfTracker(fileSystemName, accountName, this.abfsConfiguration); this.abfsCounters = abfsStoreBuilder.abfsCounters; + this.fileSystemId = abfsStoreBuilder.fileSystemId; initializeClient(uri, fileSystemName, accountName, useHttps); final Class identityTransformerClass = abfsStoreBuilder.configuration.getClass(FS_AZURE_IDENTITY_TRANSFORM_CLASS, IdentityTransformer.class, @@ -1828,6 +1830,7 @@ private AbfsClientContext populateAbfsClientContext() { new TailLatencyRequestTimeoutRetryPolicy(abfsConfiguration)) .withAbfsCounters(abfsCounters) .withAbfsPerfTracker(abfsPerfTracker) + .withFileSystemId(fileSystemId) .build(); } @@ -1979,6 +1982,7 @@ public static final class AzureBlobFileSystemStoreBuilder { private DataBlocks.BlockFactory blockFactory; private int blockOutputActiveBlocks; private BackReference fsBackRef; + private String fileSystemId; public AzureBlobFileSystemStoreBuilder withUri(URI value) { this.uri = value; @@ -2020,6 +2024,11 @@ public AzureBlobFileSystemStoreBuilder withBackReference( return this; } + public AzureBlobFileSystemStoreBuilder withFileSystemId(String fileSystemId) { + this.fileSystemId = fileSystemId; + return this; + } + public AzureBlobFileSystemStoreBuilder build() { return this; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java index 2057cd3e9fb8f..918997ab43b01 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java @@ -137,6 +137,9 @@ public final class AbfsHttpConstants { public static final String HASH = "#"; public static final String TRUE = "true"; public static final String ZERO = "0"; + public static final String UNDERSCORE = "_"; + public static final String OPENING_SQUARE_BRACKET = "["; + public static final String CLOSING_SQUARE_BRACKET = "]"; public static final String PLUS_ENCODE = "%20"; public static final String FORWARD_SLASH_ENCODE = "%2F"; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java index 3de55adcdabf1..c5eb9235fbb54 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.azurebfs.constants; +import org.apache.hadoop.fs.azurebfs.utils.MetricFormat; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.FileSystem; @@ -69,9 +70,59 @@ public final class ConfigurationKeys { */ public static final String FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED = "fs.azure.account.expect.header.enabled"; public static final String FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME = "fs.azure.account.key"; - public static final String FS_AZURE_METRIC_ACCOUNT_NAME = "fs.azure.metric.account.name"; - public static final String FS_AZURE_METRIC_ACCOUNT_KEY = "fs.azure.metric.account.key"; - public static final String FS_AZURE_METRIC_URI = "fs.azure.metric.uri"; + + /** + * Config to set separate metrics account in case user don't want to use + * existing storage account for metrics collection. + * Value: {@value}. + */ + public static final String FS_AZURE_METRICS_ACCOUNT_NAME = "fs.azure.metrics.account.name"; + /** + * Config to set metrics account key for @FS_AZURE_METRICS_ACCOUNT_NAME. + * Value: {@value}. + */ + public static final String FS_AZURE_METRICS_ACCOUNT_KEY = "fs.azure.metrics.account.key"; + /** + * Config to set metrics format. Possible values are {@link MetricFormat} + * Value: {@value}. + */ + public static final String FS_AZURE_METRICS_FORMAT = "fs.azure.metrics.format"; + /** + * Config to enable or disable metrics collection. + * Value: {@value}. + */ + public static final String FS_AZURE_METRICS_COLLECTION_ENABLED = "fs.azure.metrics.collection.enabled"; + /** + * Config to enable or disable emitting metrics when idle time exceeds threshold. + * Value: {@value}. + */ + public static final String FS_AZURE_METRICS_SHOULD_EMIT_ON_IDLE_TIME = "fs.azure.metrics.should.emit.on.idle.time"; + /** + * Config to set threshold for emitting metrics when number of operations exceeds threshold. + * Value: {@value}. + */ + public static final String FS_AZURE_METRICS_EMIT_THRESHOLD = "fs.azure.metrics.emit.threshold"; + /** + * Config to set interval in seconds to check for threshold breach for emitting metrics. + * If the number of operations exceed threshold within this interval, metrics will be emitted. + * Value: {@value}. + */ + public static final String FS_AZURE_METRICS_EMIT_THRESHOLD_INTERVAL_SECS = "fs.azure.metrics.emit.threshold.interval.secs"; + /** + * Config to set interval in minutes for emitting metrics in regular time intervals. + * Value: {@value}. + */ + public static final String FS_AZURE_METRICS_EMIT_INTERVAL_MINS = "fs.azure.metrics.emit.interval.mins"; + /** + * Config to set maximum metrics calls per second. + * Value: {@value}. + */ + public static final String FS_AZURE_METRICS_MAX_CALLS_PER_SECOND = "fs.azure.metrics.max.calls.per.second"; + /** + * Config to enable or disable backoff retry metrics collection. + * Value: {@value}. + */ + public static final String FS_AZURE_METRICS_BACKOFF_RETRY_ENABLED = "fs.azure.metrics.backoff.retry.enabled"; public static final String FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME_REGX = "fs\\.azure\\.account\\.key\\.(.*)"; public static final String FS_AZURE_SECURE_MODE = "fs.azure.secure.mode"; @@ -231,7 +282,6 @@ public final class ConfigurationKeys { * character constraints are not satisfied. **/ public static final String FS_AZURE_CLIENT_CORRELATIONID = "fs.azure.client.correlationid"; public static final String FS_AZURE_TRACINGHEADER_FORMAT = "fs.azure.tracingheader.format"; - public static final String FS_AZURE_METRIC_FORMAT = "fs.azure.metric.format"; public static final String FS_AZURE_CLUSTER_NAME = "fs.azure.cluster.name"; public static final String FS_AZURE_CLUSTER_TYPE = "fs.azure.cluster.type"; public static final String FS_AZURE_SSL_CHANNEL_MODE_KEY = "fs.azure.ssl.channel.mode"; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java index fb336da51966d..6f76f2e033c06 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java @@ -136,6 +136,13 @@ public final class FileSystemConfigurations { public static final boolean DEFAULT_ENABLE_AUTOTHROTTLING = false; public static final int DEFAULT_METRIC_IDLE_TIMEOUT_MS = 60_000; public static final int DEFAULT_METRIC_ANALYSIS_TIMEOUT_MS = 60_000; + public static final boolean DEFAULT_METRICS_COLLECTION_ENABLED = true; + public static final boolean DEFAULT_METRICS_SHOULD_EMIT_ON_IDLE_TIME = false; + public static final long DEFAULT_METRICS_EMIT_THRESHOLD = 100_000L; + public static final long DEFAULT_METRICS_EMIT_THRESHOLD_INTERVAL_SECS = 60; + public static final long DEFAULT_METRICS_EMIT_INTERVAL_MINS = 60; + public static final int DEFAULT_METRICS_MAX_CALLS_PER_SECOND = 3; + public static final boolean DEFAULT_METRICS_BACKOFF_RETRY_ENABLED = false; public static final boolean DEFAULT_FS_AZURE_ACCOUNT_LEVEL_THROTTLING_ENABLED = true; public static final int DEFAULT_ACCOUNT_OPERATION_IDLE_TIMEOUT_MS = 60_000; public static final int DEFAULT_ANALYSIS_PERIOD_MS = 10_000; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBackoffMetrics.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBackoffMetrics.java index 84bb7b77f4a59..d7651010338ce 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBackoffMetrics.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBackoffMetrics.java @@ -33,7 +33,6 @@ import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COLON; -import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EQUAL; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.THOUSAND; import static org.apache.hadoop.fs.azurebfs.constants.MetricsConstants.DOUBLE_PRECISION_FORMAT; import static org.apache.hadoop.fs.azurebfs.constants.MetricsConstants.RETRY; @@ -75,18 +74,24 @@ * retry operations in Azure Blob File System (ABFS). */ public class AbfsBackoffMetrics extends AbstractAbfsStatisticsSource { - private static final Logger LOG = LoggerFactory.getLogger(AbfsBackoffMetrics.class); + + private static final Logger LOG = LoggerFactory.getLogger( + AbfsBackoffMetrics.class); + private static final List RETRY_LIST = Arrays.asList( - RetryValue.values()); + RetryValue.values()); + + private final boolean isRetryMetricEnabled; /** * Constructor to initialize the IOStatisticsStore with counters and gauges. */ - public AbfsBackoffMetrics() { + public AbfsBackoffMetrics(final boolean isRetryMetricEnabled) { + this.isRetryMetricEnabled = isRetryMetricEnabled; IOStatisticsStore ioStatisticsStore = iostatisticsStore() - .withCounters(getMetricNames(TYPE_COUNTER)) - .withGauges(getMetricNames(TYPE_GAUGE)) - .build(); + .withCounters(getMetricNames(TYPE_COUNTER)) + .withGauges(getMetricNames(TYPE_GAUGE)) + .build(); setIOStatistics(ioStatisticsStore); } @@ -98,15 +103,15 @@ public AbfsBackoffMetrics() { */ private String[] getMetricNames(StatisticTypeEnum type) { return Arrays.stream(AbfsBackoffMetricsEnum.values()) - .filter(backoffMetricsEnum -> backoffMetricsEnum - .getStatisticType() - .equals(type)) - .flatMap(backoffMetricsEnum -> - RETRY.equals(backoffMetricsEnum.getType()) - ? RETRY_LIST.stream().map(retryCount -> - getMetricName(backoffMetricsEnum, retryCount)) - : Stream.of(backoffMetricsEnum.getName()) - ).toArray(String[]::new); + .filter(backoffMetricsEnum -> backoffMetricsEnum + .getStatisticType() + .equals(type)) + .flatMap(backoffMetricsEnum -> + RETRY.equals(backoffMetricsEnum.getType()) + ? RETRY_LIST.stream().map(retryCount -> + getMetricName(backoffMetricsEnum, retryCount)) + : Stream.of(backoffMetricsEnum.getName()) + ).toArray(String[]::new); } /** @@ -116,7 +121,8 @@ private String[] getMetricNames(StatisticTypeEnum type) { * @param retryValue the retry value * @return the constructed metric name */ - private String getMetricName(AbfsBackoffMetricsEnum metric, RetryValue retryValue) { + private String getMetricName(AbfsBackoffMetricsEnum metric, + RetryValue retryValue) { if (metric == null) { LOG.error("ABFS Backoff Metric should not be null"); return EMPTY_STRING; @@ -134,15 +140,16 @@ private String getMetricName(AbfsBackoffMetricsEnum metric, RetryValue retryValu * @param retryValue the retry value * @return the value of the metric */ - public long getMetricValue(AbfsBackoffMetricsEnum metric, RetryValue retryValue) { + public long getMetricValue(AbfsBackoffMetricsEnum metric, + RetryValue retryValue) { String metricName = getMetricName(metric, retryValue); switch (metric.getStatisticType()) { - case TYPE_COUNTER: - return lookupCounterValue(metricName); - case TYPE_GAUGE: - return lookupGaugeValue(metricName); - default: - return 0; + case TYPE_COUNTER: + return lookupCounterValue(metricName); + case TYPE_GAUGE: + return lookupGaugeValue(metricName); + default: + return 0; } } @@ -162,18 +169,19 @@ public long getMetricValue(AbfsBackoffMetricsEnum metric) { * @param metric the metric enum * @param retryValue the retry value */ - public void incrementMetricValue(AbfsBackoffMetricsEnum metric, RetryValue retryValue) { + public void incrementMetricValue(AbfsBackoffMetricsEnum metric, + RetryValue retryValue) { String metricName = getMetricName(metric, retryValue); switch (metric.getStatisticType()) { - case TYPE_COUNTER: - incCounterValue(metricName); - break; - case TYPE_GAUGE: - incGaugeValue(metricName); - break; - default: - // Do nothing - break; + case TYPE_COUNTER: + incCounterValue(metricName); + break; + case TYPE_GAUGE: + incGaugeValue(metricName); + break; + default: + // Do nothing + break; } } @@ -193,18 +201,20 @@ public void incrementMetricValue(AbfsBackoffMetricsEnum metric) { * @param value the new value of the metric * @param retryValue the retry value */ - public void setMetricValue(AbfsBackoffMetricsEnum metric, long value, RetryValue retryValue) { + public void setMetricValue(AbfsBackoffMetricsEnum metric, + long value, + RetryValue retryValue) { String metricName = getMetricName(metric, retryValue); switch (metric.getStatisticType()) { - case TYPE_COUNTER: - setCounterValue(metricName, value); - break; - case TYPE_GAUGE: - setGaugeValue(metricName, value); - break; - default: - // Do nothing - break; + case TYPE_COUNTER: + setCounterValue(metricName, value); + break; + case TYPE_GAUGE: + setGaugeValue(metricName, value); + break; + default: + // Do nothing + break; } } @@ -227,9 +237,10 @@ public void setMetricValue(AbfsBackoffMetricsEnum metric, long value) { * @return String metrics value with precision */ private String getPrecisionMetrics(AbfsBackoffMetricsEnum metricName, - RetryValue retryCount, - long denominator) { - return format(DOUBLE_PRECISION_FORMAT, (double) getMetricValue(metricName, retryCount) / denominator); + RetryValue retryCount, + long denominator) { + return format(DOUBLE_PRECISION_FORMAT, + (double) getMetricValue(metricName, retryCount) / denominator); } /** @@ -240,25 +251,24 @@ private String getPrecisionMetrics(AbfsBackoffMetricsEnum metricName, private void getRetryMetrics(StringBuilder metricBuilder) { for (RetryValue retryCount : RETRY_LIST) { long totalRequests = getMetricValue(TOTAL_REQUESTS, retryCount); - metricBuilder.append(REQUEST_COUNT) - .append(retryCount.getValue()) - .append(REQUESTS) - .append(getMetricValue(NUMBER_OF_REQUESTS_SUCCEEDED, retryCount)); + if (getMetricValue(NUMBER_OF_REQUESTS_SUCCEEDED, retryCount) > 0) { + metricBuilder.append(REQUEST_COUNT) + .append(retryCount.getValue()) + .append(REQUESTS) + .append(getMetricValue(NUMBER_OF_REQUESTS_SUCCEEDED, retryCount)); + } if (totalRequests > 0) { metricBuilder.append(MIN_MAX_AVERAGE) - .append(retryCount.getValue()) - .append(REQUESTS) - .append(getPrecisionMetrics(MIN_BACK_OFF, retryCount, THOUSAND)) - .append(SECONDS) - .append(getPrecisionMetrics(MAX_BACK_OFF, retryCount, THOUSAND)) - .append(SECONDS) - .append(getPrecisionMetrics(TOTAL_BACK_OFF, retryCount, totalRequests * THOUSAND)) - .append(SECONDS); - } else { - metricBuilder.append(MIN_MAX_AVERAGE) - .append(retryCount.getValue()) - .append(REQUESTS + EQUAL + 0 + SECONDS); + .append(retryCount.getValue()) + .append(REQUESTS) + .append(getPrecisionMetrics(MIN_BACK_OFF, retryCount, THOUSAND)) + .append(SECONDS) + .append(getPrecisionMetrics(MAX_BACK_OFF, retryCount, THOUSAND)) + .append(SECONDS) + .append(getPrecisionMetrics(TOTAL_BACK_OFF, retryCount, + totalRequests * THOUSAND)) + .append(SECONDS); } } } @@ -269,29 +279,42 @@ private void getRetryMetrics(StringBuilder metricBuilder) { * @param metricBuilder the string builder to append the metrics */ private void getBaseMetrics(StringBuilder metricBuilder) { - long totalRequestsThrottled = getMetricValue(NUMBER_OF_NETWORK_FAILED_REQUESTS) + long totalRequestsThrottled = + getMetricValue(NUMBER_OF_NETWORK_FAILED_REQUESTS) + getMetricValue(NUMBER_OF_IOPS_THROTTLED_REQUESTS) + getMetricValue(NUMBER_OF_OTHER_THROTTLED_REQUESTS) + getMetricValue(NUMBER_OF_BANDWIDTH_THROTTLED_REQUESTS); - metricBuilder.append(BANDWIDTH_THROTTLED_REQUESTS) - .append(getMetricValue(NUMBER_OF_BANDWIDTH_THROTTLED_REQUESTS)) - .append(IOPS_THROTTLED_REQUESTS) - .append(getMetricValue(NUMBER_OF_IOPS_THROTTLED_REQUESTS)) - .append(OTHER_THROTTLED_REQUESTS) - .append(getMetricValue(NUMBER_OF_OTHER_THROTTLED_REQUESTS)) - .append(PERCENTAGE_THROTTLED_REQUESTS) - .append(formatPercent(totalRequestsThrottled/ (double) getMetricValue(TOTAL_NUMBER_OF_REQUESTS), 3)) - .append(NETWORK_ERROR_REQUESTS) - .append(getMetricValue(NUMBER_OF_NETWORK_FAILED_REQUESTS)) - .append(SUCCESS_REQUESTS_WITHOUT_RETRY) - .append(getMetricValue(NUMBER_OF_REQUESTS_SUCCEEDED_WITHOUT_RETRYING)) - .append(FAILED_REQUESTS) - .append(getMetricValue(NUMBER_OF_REQUESTS_FAILED)) - .append(TOTAL_REQUESTS_COUNT) - .append(getMetricValue(TOTAL_NUMBER_OF_REQUESTS)) - .append(MAX_RETRY) - .append(getMetricValue(MAX_RETRY_COUNT)); + appendIfPositive(metricBuilder, BANDWIDTH_THROTTLED_REQUESTS, + getMetricValue(NUMBER_OF_BANDWIDTH_THROTTLED_REQUESTS)); + + appendIfPositive(metricBuilder, IOPS_THROTTLED_REQUESTS, + getMetricValue(NUMBER_OF_IOPS_THROTTLED_REQUESTS)); + + appendIfPositive(metricBuilder, OTHER_THROTTLED_REQUESTS, + getMetricValue(NUMBER_OF_OTHER_THROTTLED_REQUESTS)); + + // For percentage, we always want it (even if 0%) + if (totalRequestsThrottled > 0) { + appendAlways(metricBuilder, PERCENTAGE_THROTTLED_REQUESTS, + formatPercent(totalRequestsThrottled / (double) getMetricValue( + TOTAL_NUMBER_OF_REQUESTS), 3)); + } + + appendIfPositive(metricBuilder, NETWORK_ERROR_REQUESTS, + getMetricValue(NUMBER_OF_NETWORK_FAILED_REQUESTS)); + + appendIfPositive(metricBuilder, SUCCESS_REQUESTS_WITHOUT_RETRY, + getMetricValue(NUMBER_OF_REQUESTS_SUCCEEDED_WITHOUT_RETRYING)); + + appendIfPositive(metricBuilder, FAILED_REQUESTS, + getMetricValue(NUMBER_OF_REQUESTS_FAILED)); + + appendIfPositive(metricBuilder, TOTAL_REQUESTS_COUNT, + getMetricValue(TOTAL_NUMBER_OF_REQUESTS)); + + appendIfPositive(metricBuilder, MAX_RETRY, + getMetricValue(MAX_RETRY_COUNT)); } /** @@ -305,7 +328,9 @@ public String toString() { return EMPTY_STRING; } StringBuilder metricBuilder = new StringBuilder(); - getRetryMetrics(metricBuilder); + if (isRetryMetricEnabled) { + getRetryMetrics(metricBuilder); + } getBaseMetrics(metricBuilder); return metricBuilder.toString(); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index b6a0e1e1ce2d0..db3e163ca4d5f 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -25,12 +25,10 @@ import java.io.UnsupportedEncodingException; import java.lang.reflect.InvocationTargetException; import java.net.HttpURLConnection; -import java.net.InetAddress; import java.net.MalformedURLException; import java.net.URI; import java.net.URL; import java.net.URLEncoder; -import java.net.UnknownHostException; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; import java.util.ArrayList; @@ -38,12 +36,9 @@ import java.util.Hashtable; import java.util.List; import java.util.Locale; -import java.util.Timer; -import java.util.TimerTask; import java.util.concurrent.Callable; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -86,9 +81,8 @@ import org.apache.hadoop.fs.azurebfs.security.ContextEncryptionAdapter; import org.apache.hadoop.fs.azurebfs.utils.DateTimeUtils; import org.apache.hadoop.fs.azurebfs.utils.EncryptionType; -import org.apache.hadoop.fs.azurebfs.utils.MetricFormat; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; -import org.apache.hadoop.fs.azurebfs.utils.UriUtils; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat; import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.store.LogExactlyOnce; @@ -158,7 +152,6 @@ public abstract class AbfsClient implements Closeable { public static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class); public static final String HUNDRED_CONTINUE_USER_AGENT = SINGLE_WHITE_SPACE + HUNDRED_CONTINUE + SEMICOLON; - public static final String ABFS_CLIENT_TIMER_THREAD_NAME = "abfs-timer-client"; public static final String FNS_BLOB_USER_AGENT_IDENTIFIER = "FNS"; private final URL baseUrl; @@ -182,13 +175,6 @@ public abstract class AbfsClient implements Closeable { private AccessTokenProvider tokenProvider; private SASTokenProvider sasTokenProvider; private final AbfsCounters abfsCounters; - private Timer timer; - private final String abfsMetricUrl; - private boolean isMetricCollectionEnabled = false; - private final MetricFormat metricFormat; - private final AtomicBoolean isMetricCollectionStopped; - private final int metricAnalysisPeriod; - private final int metricIdlePeriod; private EncryptionContextProvider encryptionContextProvider = null; private EncryptionType encryptionType = EncryptionType.NONE; private final AbfsThrottlingIntercept intercept; @@ -197,14 +183,15 @@ public abstract class AbfsClient implements Closeable { private final ListeningScheduledExecutorService executorService; private boolean renameResilience; - private TimerTask runningTimerTask; - private boolean isSendMetricCall; - private SharedKeyCredentials metricSharedkeyCredentials = null; private KeepAliveCache keepAliveCache; private AbfsApacheHttpClient abfsApacheHttpClient; + private final AbfsMetricsManager abfsMetricsManager; + + private final String fileSystemId; + private AbfsServiceType abfsServiceType; /** @@ -221,7 +208,8 @@ private AbfsClient(final URL baseUrl, this.baseUrl = baseUrl; this.sharedKeyCredentials = sharedKeyCredentials; String baseUrlString = baseUrl.toString(); - this.filesystem = baseUrlString.substring(baseUrlString.lastIndexOf(FORWARD_SLASH) + 1); + int indexLastForwardSlash = baseUrlString.lastIndexOf(FORWARD_SLASH); + this.filesystem = baseUrlString.substring(indexLastForwardSlash + 1); this.abfsConfiguration = abfsConfiguration; this.exponentialRetryPolicy = abfsClientContext.getExponentialRetryPolicy(); this.staticRetryPolicy = abfsClientContext.getStaticRetryPolicy(); @@ -231,6 +219,7 @@ private AbfsClient(final URL baseUrl, this.intercept = AbfsThrottlingInterceptFactory.getInstance(accountName, abfsConfiguration); this.tailLatencyTracker = AbfsTailLatencyTrackerFactory.getInstance(accountName, abfsConfiguration); this.renameResilience = abfsConfiguration.getRenameResilience(); + this.fileSystemId = abfsClientContext.getFileSystemId(); this.abfsServiceType = abfsServiceType; if (encryptionContextProvider != null) { @@ -281,42 +270,17 @@ private AbfsClient(final URL baseUrl, new ThreadFactoryBuilder().setNameFormat("AbfsClient Lease Ops").setDaemon(true).build(); this.executorService = MoreExecutors.listeningDecorator( HadoopExecutors.newScheduledThreadPool(this.abfsConfiguration.getNumLeaseThreads(), tf)); - this.metricFormat = abfsConfiguration.getMetricFormat(); - this.isMetricCollectionStopped = new AtomicBoolean(false); - this.metricAnalysisPeriod = abfsConfiguration.getMetricAnalysisTimeout(); - this.metricIdlePeriod = abfsConfiguration.getMetricIdleTimeout(); - if (StringUtils.isNotEmpty(metricFormat.toString())) { - String metricAccountName = abfsConfiguration.getMetricAccount(); - String metricAccountKey = abfsConfiguration.getMetricAccountKey(); - if (StringUtils.isNotEmpty(metricAccountName) && StringUtils.isNotEmpty(metricAccountKey)) { - isMetricCollectionEnabled = true; - abfsCounters.initializeMetrics(metricFormat); - int dotIndex = metricAccountName.indexOf(AbfsHttpConstants.DOT); - if (dotIndex <= 0) { - throw new InvalidUriException( - metricAccountName + " - account name is not fully qualified."); - } - try { - metricSharedkeyCredentials = new SharedKeyCredentials( - metricAccountName.substring(0, dotIndex), - metricAccountKey); - } catch (IllegalArgumentException e) { - throw new IOException("Exception while initializing metric credentials ", e); - } - } - } - if (isMetricCollectionEnabled) { - this.timer = new Timer( - ABFS_CLIENT_TIMER_THREAD_NAME, true); - timer.schedule(new TimerTaskImpl(), - metricIdlePeriod, - metricIdlePeriod); - } + + this.abfsMetricsManager = new AbfsMetricsManager(abfsConfiguration, abfsCounters, + baseUrlString, indexLastForwardSlash, accountName, fileSystemId); + // register the client to Aggregated Metrics Manager + abfsMetricsManager.getAggregateMetricsManager() + .registerClient(accountName, this); + // Initialize write thread pool metrics if dynamic write thread pool scaling is enabled. if (abfsConfiguration.isDynamicWriteThreadPoolEnablement()) { abfsCounters.initializeWriteResourceUtilizationMetrics(); } - this.abfsMetricUrl = abfsConfiguration.getMetricUri(); // Initialize read thread pool metrics if ReadAheadV2 and its dynamic scaling feature are enabled. if (abfsConfiguration.isReadAheadV2Enabled() && abfsConfiguration.isReadAheadV2DynamicScalingEnabled()) { abfsCounters.initializeReadResourceUtilizationMetrics(); @@ -329,7 +293,7 @@ private AbfsClient(final URL baseUrl, Configuration.class).newInstance(abfsConfiguration.getRawConfiguration()); } catch (IllegalAccessException | InstantiationException | IllegalArgumentException | InvocationTargetException | NoSuchMethodException e) { - LOG.error("IdentityTransformer Init Falied", e); + LOG.error("IdentityTransformer Init Failed", e); throw new IOException(e); } LOG.trace("IdentityTransformer init complete"); @@ -382,9 +346,10 @@ public AbfsClient(final URL baseUrl, @Override public void close() throws IOException { - if (isMetricCollectionEnabled && runningTimerTask != null) { - runningTimerTask.cancel(); - timer.cancel(); + if (abfsMetricsManager != null) { + abfsMetricsManager.close(); + abfsMetricsManager.getAggregateMetricsManager() + .deregisterClient(accountName, this); } if (keepAliveCache != null) { keepAliveCache.close(); @@ -441,10 +406,6 @@ SharedKeyCredentials getSharedKeyCredentials() { return sharedKeyCredentials; } - SharedKeyCredentials getMetricSharedkeyCredentials() { - return metricSharedkeyCredentials; - } - public void setEncryptionType(EncryptionType encryptionType) { this.encryptionType = encryptionType; } @@ -465,6 +426,14 @@ AbfsThrottlingIntercept getIntercept() { return intercept; } + /** + * @return AbfsMetricsManager instance. + */ + @VisibleForTesting + public AbfsMetricsManager getAbfsMetricsManager() { + return abfsMetricsManager; + } + /** * Create request headers for Rest Operation using the current API version. * @return default request headers @@ -641,8 +610,8 @@ public abstract AbfsRestOperation deleteFilesystem(TracingContext tracingContext * the path. * @param contextEncryptionAdapter: object that contains the encryptionContext and * encryptionKey created from the developer provided implementation of - * {@link org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider} - * @param tracingContext: Object of {@link org.apache.hadoop.fs.azurebfs.utils.TracingContext} + * {@link EncryptionContextProvider} + * @param tracingContext: Object of {@link TracingContext} * correlating to the current fs.create() request. * @return object of {@link AbfsRestOperation} which contain all the information * about the communication with the server. The information is in @@ -1575,128 +1544,41 @@ protected AccessTokenProvider getTokenProvider() { return tokenProvider; } - /** - * Retrieves a TracingContext object configured for metric tracking. - * This method creates a TracingContext object with the validated client correlation ID, - * the host name of the local machine (or "UnknownHost" if unable to determine), - * the file system operation type set to GET_ATTR, and additional configuration parameters - * for metric tracking. - * The TracingContext is intended for use in tracking metrics related to Azure Blob FileSystem (ABFS) operations. - * - * @return A TracingContext object configured for metric tracking. - */ - private TracingContext getMetricTracingContext() { - String hostName; - try { - hostName = InetAddress.getLocalHost().getHostName(); - } catch (UnknownHostException e) { - hostName = "UnknownHost"; - } - return new TracingContext(TracingContext.validateClientCorrelationID( - abfsConfiguration.getClientCorrelationId()), - hostName, FSOperationType.GET_ATTR, true, - abfsConfiguration.getTracingHeaderFormat(), - null, abfsCounters.toString()); - } - - /** - * Synchronized method to suspend or resume timer. - * @param timerFunctionality resume or suspend. - * @param timerTask The timertask object. - * @return true or false. - */ - boolean timerOrchestrator(TimerFunctionality timerFunctionality, TimerTask timerTask) { - switch (timerFunctionality) { - case RESUME: - if (isMetricCollectionEnabled && isMetricCollectionStopped.get()) { - synchronized (this) { - if (isMetricCollectionStopped.get()) { - resumeTimer(); - } - } - } - break; - case SUSPEND: - long now = System.currentTimeMillis(); - long lastExecutionTime = abfsCounters.getLastExecutionTime().get(); - if (isMetricCollectionEnabled && (now - lastExecutionTime >= metricAnalysisPeriod)) { - synchronized (this) { - if (!isMetricCollectionStopped.get()) { - timerTask.cancel(); - timer.purge(); - isMetricCollectionStopped.set(true); - return true; - } - } - } - break; - default: - break; - } - return false; - } - - private void resumeTimer() { - isMetricCollectionStopped.set(false); - timer.schedule(new TimerTaskImpl(), - metricIdlePeriod, - metricIdlePeriod); - } - /** * Initiates a metric call to the Azure Blob FileSystem (ABFS) for retrieving file system properties. * This method performs a HEAD request to the specified metric URL, using default headers and query parameters. * - * @param tracingContext The tracing context to be used for capturing tracing information. + * @param metricsData The tracing context to be used for capturing tracing information. * @throws IOException throws IOException. */ - public void getMetricCall(TracingContext tracingContext) throws IOException { - this.isSendMetricCall = true; + public void getMetricCall(String metricsData) throws IOException { + if (StringUtils.isEmpty(metricsData)) { + return; + } final List requestHeaders = createDefaultHeaders(); - final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + final AbfsUriQueryBuilder abfsUriQueryBuilder + = createDefaultUriQueryBuilder(); abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); // Construct the URL for the metric call // In case of blob storage, the URL is changed to DFS URL - final URL url = UriUtils.changeUrlFromBlobToDfs( - createRequestUrl(new URL(abfsMetricUrl), - EMPTY_STRING, abfsUriQueryBuilder.toString())); + final URL url = createRequestUrl(abfsMetricsManager.getMetricsUrl(), + EMPTY_STRING, abfsUriQueryBuilder.toString()); final AbfsRestOperation op = getAbfsRestOperation( - AbfsRestOperationType.GetFileSystemProperties, - HTTP_METHOD_HEAD, - url, - requestHeaders); + AbfsRestOperationType.GetFileSystemProperties, + HTTP_METHOD_HEAD, + url, + requestHeaders); + TracingContext tracingContext = new TracingContext( + TracingContext.validateClientCorrelationID( + abfsConfiguration.getClientCorrelationId()), + fileSystemId, FSOperationType.GET_ATTR, true, + TracingHeaderFormat.AGGREGATED_METRICS_FORMAT, + null, metricsData); try { op.execute(tracingContext); - } finally { - this.isSendMetricCall = false; - } - } - - public boolean isSendMetricCall() { - return isSendMetricCall; - } - - public boolean isMetricCollectionEnabled() { - return isMetricCollectionEnabled; - } - - class TimerTaskImpl extends TimerTask { - TimerTaskImpl() { - runningTimerTask = this; - } - @Override - public void run() { - try { - if (timerOrchestrator(TimerFunctionality.SUSPEND, this)) { - try { - getMetricCall(getMetricTracingContext()); - } finally { - abfsCounters.initializeMetrics(metricFormat); - } - } - } catch (IOException e) { - } + } catch (AzureBlobFileSystemException e) { + // Ignore the exception and continue. } } @@ -1791,11 +1673,6 @@ KeepAliveCache getKeepAliveCache() { return keepAliveCache; } - @VisibleForTesting - protected Timer getTimer() { - return timer; - } - protected String getUserAgent() { return userAgent; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientContext.java index 0744d987a4dda..27b2d5996e02e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientContext.java @@ -29,18 +29,20 @@ public class AbfsClientContext { private final TailLatencyRequestTimeoutRetryPolicy tailLatencyRequestTimeoutRetryPolicy; private final AbfsPerfTracker abfsPerfTracker; private final AbfsCounters abfsCounters; + private final String fileSystemId; AbfsClientContext( ExponentialRetryPolicy exponentialRetryPolicy, StaticRetryPolicy staticRetryPolicy, TailLatencyRequestTimeoutRetryPolicy tailLatencyRequestTimeoutRetryPolicy, AbfsPerfTracker abfsPerfTracker, - AbfsCounters abfsCounters) { + AbfsCounters abfsCounters, String fileSystemId) { this.exponentialRetryPolicy = exponentialRetryPolicy; this.staticRetryPolicy = staticRetryPolicy; this.tailLatencyRequestTimeoutRetryPolicy = tailLatencyRequestTimeoutRetryPolicy; this.abfsPerfTracker = abfsPerfTracker; this.abfsCounters = abfsCounters; + this.fileSystemId = fileSystemId; } public ExponentialRetryPolicy getExponentialRetryPolicy() { @@ -66,4 +68,8 @@ public AbfsPerfTracker getAbfsPerfTracker() { AbfsCounters getAbfsCounters() { return abfsCounters; } + + public String getFileSystemId() { + return fileSystemId; + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientContextBuilder.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientContextBuilder.java index 3e02a49985884..5a175d2726301 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientContextBuilder.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientContextBuilder.java @@ -29,6 +29,7 @@ public class AbfsClientContextBuilder { private TailLatencyRequestTimeoutRetryPolicy tailLatencyRequestTimeoutRetryPolicy; private AbfsPerfTracker abfsPerfTracker; private AbfsCounters abfsCounters; + private String fileSystemId; public AbfsClientContextBuilder withExponentialRetryPolicy( final ExponentialRetryPolicy exponentialRetryPolicy) { @@ -59,6 +60,11 @@ public AbfsClientContextBuilder withAbfsCounters(final AbfsCounters abfsCounters return this; } + public AbfsClientContextBuilder withFileSystemId(final String fileSystemId) { + this.fileSystemId = fileSystemId; + return this; + } + /** * Build the context and get the instance with the properties selected. * @@ -71,6 +77,6 @@ public AbfsClientContext build() { staticRetryPolicy, tailLatencyRequestTimeoutRetryPolicy, abfsPerfTracker, - abfsCounters); + abfsCounters, fileSystemId); } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsCounters.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsCounters.java index 4512db98fcc69..8ada20abf4358 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsCounters.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsCounters.java @@ -25,6 +25,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; import org.apache.hadoop.fs.azurebfs.AbfsStatistic; import org.apache.hadoop.fs.azurebfs.utils.MetricFormat; import org.apache.hadoop.fs.statistics.DurationTracker; @@ -77,7 +78,8 @@ String formString(String prefix, String separator, String suffix, @Override DurationTracker trackDuration(String key); - void initializeMetrics(MetricFormat metricFormat); + void initializeMetrics(MetricFormat metricFormat, + AbfsConfiguration abfsConfiguration); AbfsBackoffMetrics getAbfsBackoffMetrics(); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsMetricsManager.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsMetricsManager.java new file mode 100644 index 0000000000000..455d71f9ed8ba --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsMetricsManager.java @@ -0,0 +1,399 @@ +/** + * 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.hadoop.fs.azurebfs.services; + +import java.io.Closeable; +import java.io.IOException; +import java.net.URL; +import java.util.Timer; +import java.util.TimerTask; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; +import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException; +import org.apache.hadoop.fs.azurebfs.utils.MetricFormat; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.azurebfs.utils.UriUtils; + +import static org.apache.commons.lang3.StringUtils.isNotEmpty; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COLON; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.FORWARD_SLASH; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.HTTPS_SCHEME; +import static org.apache.hadoop.fs.azurebfs.enums.AbfsBackoffMetricsEnum.TOTAL_NUMBER_OF_REQUESTS; +import static org.apache.hadoop.fs.azurebfs.services.AbfsClient.LOG; + +/** + * AbfsMetricsManager is responsible for managing metrics collection + * and emission for an AbfsClient instance. + */ +public class AbfsMetricsManager implements Closeable { + + // Timer thread name for AbfsMetricsManager + public static final String ABFS_CLIENT_TIMER_THREAD_NAME + = "abfs-timer-client"; + + // Timer for scheduling metric emission tasks based on idle time + private Timer timer; + + // URL for sending metrics + private URL metricUrl; + + // Shared key credentials for metric account + private SharedKeyCredentials metricSharedkeyCredentials = null; + + // Currently running TimerTask + private TimerTask runningTimerTask; + + // Metric analysis periods + private final int metricAnalysisPeriod; + + // Metric idle period + private final int metricIdlePeriod; + + // Flag to indicate if a separate metric account is used + private boolean hasSeparateMetricAccount = false; + + // Flag to indicate if metric collection is enabled + private final AtomicBoolean isMetricCollectionEnabled + = new AtomicBoolean(false); + + // Metric format for metrics + private MetricFormat metricFormat; + + // Flag to indicate if metric collection is stopped + private final AtomicBoolean isMetricCollectionStopped; + + // AggregateMetricsManager instance + private final AggregateMetricsManager aggregateMetricsManager; + + // Scheduler to emit aggregated metric based on time + private ScheduledExecutorService metricsEmitScheduler = null; + + // AbfsConfiguration instance + private final AbfsConfiguration abfsConfiguration; + + // AbfsCounters instance + private final AbfsCounters abfsCounters; + + // File system ID + private final String fileSystemId; + + // Storage account name + private final String accountName; + + /** + * Constructor for AbfsMetricsManager. + * + * @param abfsConfiguration AbfsConfiguration object. + * @param abfsCounters AbfsCounters object. + * @param baseUrlString Base URL string of the AbfsClient. + * @param indexLastForwardSlash Index of last forward slash in the base URL string. + * @param accountName Storage account name. + * @param fileSystemId File system ID. + */ + public AbfsMetricsManager(final AbfsConfiguration abfsConfiguration, + final AbfsCounters abfsCounters, final String baseUrlString, + final int indexLastForwardSlash, final String accountName, + final String fileSystemId) { + this.abfsConfiguration = abfsConfiguration; + this.abfsCounters = abfsCounters; + this.fileSystemId = fileSystemId; + this.isMetricCollectionEnabled.set( + abfsConfiguration.isMetricsCollectionEnabled()); + this.isMetricCollectionStopped = new AtomicBoolean(false); + this.aggregateMetricsManager = AggregateMetricsManager.getInstance( + abfsConfiguration.getMetricsEmitIntervalInMins(), + abfsConfiguration.getMaxMetricsCallsPerSecond()); + this.metricAnalysisPeriod = abfsConfiguration.getMetricAnalysisTimeout(); + this.metricIdlePeriod = abfsConfiguration.getMetricIdleTimeout(); + this.accountName = accountName; + if (isMetricCollectionEnabled()) { + try { + String metricAccountName = abfsConfiguration.getMetricAccount(); + String metricAccountKey = abfsConfiguration.getMetricAccountKey(); + this.metricFormat = abfsConfiguration.getMetricFormat(); + if (isNotEmpty(metricAccountName) && isNotEmpty( + metricAccountKey)) { + int dotIndex = metricAccountName.indexOf(AbfsHttpConstants.DOT); + if (dotIndex <= 0) { + throw new InvalidUriException( + metricAccountName + " - account name is not fully qualified."); + } + try { + metricSharedkeyCredentials = new SharedKeyCredentials( + metricAccountName.substring(0, dotIndex), + metricAccountKey); + hasSeparateMetricAccount = true; + setMetricsUrl(metricAccountName.startsWith(HTTPS_SCHEME) + ? metricAccountName : HTTPS_SCHEME + COLON + + FORWARD_SLASH + FORWARD_SLASH + metricAccountName); + } catch (IllegalArgumentException e) { + throw new IOException( + "Exception while initializing metric credentials ", e); + } + } else { + setMetricsUrl(baseUrlString.substring(0, indexLastForwardSlash + 1)); + } + // Once the metric URL is set, initialize the metrics + abfsCounters.initializeMetrics(metricFormat, abfsConfiguration); + // Metrics emitter scheduler + this.metricsEmitScheduler + = Executors.newSingleThreadScheduledExecutor(); + // run every 1 minute to check the metrics count + this.metricsEmitScheduler.scheduleWithFixedDelay( + () -> { + if (abfsCounters.getAbfsBackoffMetrics() + .getMetricValue(TOTAL_NUMBER_OF_REQUESTS) + >= abfsConfiguration.getMetricsEmitThreshold()) { + emitCollectedMetrics(); + } + }, + abfsConfiguration.getMetricsEmitThresholdIntervalInSecs(), + abfsConfiguration.getMetricsEmitThresholdIntervalInSecs(), + TimeUnit.SECONDS); + + // run every metricInterval minutes + this.metricsEmitScheduler.scheduleWithFixedDelay( + this::emitCollectedMetrics, + abfsConfiguration.getMetricsEmitIntervalInMins(), + abfsConfiguration.getMetricsEmitIntervalInMins(), + TimeUnit.MINUTES); + + // emit metrics based on idea time + if (abfsConfiguration.shouldEmitMetricsOnIdleTime()) { + this.timer = new Timer( + ABFS_CLIENT_TIMER_THREAD_NAME, true); + timer.schedule(new TimerTaskImpl(), + metricIdlePeriod, + metricIdlePeriod); + } + } catch (Exception e) { + LOG.error("Metrics disabled. Failed to initialize metrics for {}", + baseUrlString, e); + this.isMetricCollectionEnabled.set(false); + } + } + } + + /** + * Closes the metrics resources. + * This method cancels any running timer tasks, shuts down the metrics emission scheduler, + * and emits any collected metrics before closing. + */ + @Override + public void close() { + if (runningTimerTask != null) { + runningTimerTask.cancel(); + runningTimerTask = null; + } + if (timer != null) { + timer.cancel(); + timer = null; + } + if (metricsEmitScheduler != null && !metricsEmitScheduler.isShutdown()) { + metricsEmitScheduler.shutdownNow(); + metricsEmitScheduler = null; + } + if (isMetricCollectionEnabled()) { + emitCollectedMetrics(); + } + } + + /** + * Retrieves a TracingContext object configured for metric tracking. + * This method creates a TracingContext object with the validated client correlation ID, + * the host name of the local machine (or "UnknownHost" if unable to determine), + * the file system operation type set to GET_ATTR, and additional configuration parameters + * for metric tracking. + * The TracingContext is intended for use in tracking metrics related to Azure Blob FileSystem (ABFS) operations. + * + * @return A TracingContext object configured for metric tracking. + */ + private synchronized String getMetricsData() { + String metrics = abfsCounters.toString(); + if (StringUtils.isEmpty(metrics)) { + return null; + } + abfsCounters.initializeMetrics(metricFormat, abfsConfiguration); + return TracingContext.validateClientCorrelationID( + abfsConfiguration.getClientCorrelationId()) + COLON + fileSystemId + + COLON + metrics; + } + + /** + * Synchronized method to suspend or resume timer. + * @param timerFunctionality resume or suspend. + * @param timerTask The timertask object. + * @return true or false. + */ + boolean timerOrchestrator(TimerFunctionality timerFunctionality, + TimerTask timerTask) { + switch (timerFunctionality) { + case RESUME: + if (isMetricCollectionEnabled() && isMetricCollectionStopped.get()) { + synchronized (this) { + if (isMetricCollectionStopped.get()) { + resumeTimer(); + } + } + } + break; + case SUSPEND: + long now = System.currentTimeMillis(); + long lastExecutionTime = abfsCounters.getLastExecutionTime().get(); + if (isMetricCollectionEnabled() && (now - lastExecutionTime + >= metricAnalysisPeriod)) { + synchronized (this) { + if (!isMetricCollectionStopped.get()) { + timerTask.cancel(); + timer.purge(); + isMetricCollectionStopped.set(true); + return true; + } + } + } + break; + default: + break; + } + return false; + } + + /** + * Resumes the timer for metric collection. + * This method sets the isMetricCollectionStopped flag to false + * and schedules a new TimerTaskImpl to run at fixed intervals + * defined by the metricIdlePeriod. + */ + private void resumeTimer() { + isMetricCollectionStopped.set(false); + timer.schedule(new TimerTaskImpl(), + metricIdlePeriod, + metricIdlePeriod); + } + + /** + * Checks if metric collection is enabled. + * + * @return true if metric collection is enabled, false otherwise. + */ + public boolean isMetricCollectionEnabled() { + return isMetricCollectionEnabled.get() && fileSystemId != null; + } + + /** + * Getter for metric URL. + * + * @return metricUrl + */ + @VisibleForTesting + public URL getMetricsUrl() { + return metricUrl; + } + + /** + * Setter for metric URL. + * Converts blob URL to dfs URL in case of blob storage account. + * + * @param urlString to be set as metricUrl. + * @throws IOException if URL is malformed. + */ + private void setMetricsUrl(String urlString) throws IOException { + metricUrl = UriUtils.changeUrlFromBlobToDfs(new URL(urlString)); + } + + /** + * TimerTask implementation for emitting collected metrics based on ideal time. + * This class extends TimerTask and overrides the run method to + * check if the timer should be suspended based on the configured + * metric analysis period. If the timer is suspended, it triggers + * the emission of collected metrics. + */ + class TimerTaskImpl extends TimerTask { + + TimerTaskImpl() { + runningTimerTask = this; + } + + @Override + public void run() { + if (timerOrchestrator(TimerFunctionality.SUSPEND, this)) { + emitCollectedMetrics(); + } + } + } + + /** + * Emits the collected metrics by making a metric call to the Azure Blob FileSystem (ABFS). + * This method checks if metric collection is enabled and, if so, attempts to perform + * a metric call using the configured tracing context. Any IOException encountered during + * the metric call is logged and ignored to prevent termination of the timer task. + * Finally, it re-initializes the metrics in the AbfsCounters instance using the specified + * metric format. + */ + public void emitCollectedMetrics() { + if (!isMetricCollectionEnabled()) { + return; + } + this.aggregateMetricsManager.recordMetric(accountName, getMetricsData()); + } + + /** + * Getter for timer. + */ + @VisibleForTesting + protected Timer getTimer() { + return timer; + } + + /** + * Getter for metricsEmitScheduler. + */ + @VisibleForTesting + ScheduledExecutorService getMetricsEmitScheduler() { + return metricsEmitScheduler; + } + + /** + * @return true if metric account name and key are different from storage account. + */ + public boolean hasSeparateMetricAccount() { + return hasSeparateMetricAccount; + } + + /** + * Getter for metric shared key credentials. + */ + public SharedKeyCredentials getMetricSharedkeyCredentials() { + return metricSharedkeyCredentials; + } + + /** + * Getter for AggregateMetricsManager. + */ + public AggregateMetricsManager getAggregateMetricsManager() { + return aggregateMetricsManager; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsReadFooterMetrics.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsReadFooterMetrics.java index d22f0ac4985dd..84898954e5f4b 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsReadFooterMetrics.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsReadFooterMetrics.java @@ -15,6 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.hadoop.fs.azurebfs.services; import java.util.List; @@ -36,6 +37,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CHAR_UNDERSCORE; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COLON; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.UNDERSCORE; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_KB; import static org.apache.hadoop.fs.azurebfs.constants.MetricsConstants.CHAR_DOLLAR; import static org.apache.hadoop.fs.azurebfs.constants.MetricsConstants.DOUBLE_PRECISION_FORMAT; @@ -62,476 +64,538 @@ * This class is responsible for tracking and updating metrics related to reading footers in files. */ public class AbfsReadFooterMetrics extends AbstractAbfsStatisticsSource { - private static final Logger LOG = LoggerFactory.getLogger(AbfsReadFooterMetrics.class); - private static final String FOOTER_LENGTH = "20"; - private static final List FILE_TYPE_LIST = - Arrays.asList(FileType.values()); - private final Map fileTypeMetricsMap = - new ConcurrentHashMap<>(); + + private static final Logger LOG = LoggerFactory.getLogger( + AbfsReadFooterMetrics.class); + + // Footer length in KB to identify Parquet files. + private static final String FOOTER_LENGTH = "20"; + + private static final List FILE_TYPE_LIST = + Arrays.asList(FileType.values()); + + private Map fileTypeMetricsMap = + new ConcurrentHashMap<>(); + + /** + * Inner class to handle file type checks. + */ + private static final class FileTypeMetrics { + + private final AtomicBoolean collectMetrics; + + private final AtomicBoolean collectMetricsForNextRead; + + private final AtomicBoolean collectLenMetrics; + + private final AtomicLong readCount; + + private final AtomicLong offsetOfFirstRead; + + private FileType fileType = null; + + private String sizeReadByFirstRead; + + private String offsetDiffBetweenFirstAndSecondRead; /** - * Inner class to handle file type checks. + * Constructor to initialize the file type metrics. */ - private static final class FileTypeMetrics { - private final AtomicBoolean collectMetrics; - private final AtomicBoolean collectMetricsForNextRead; - private final AtomicBoolean collectLenMetrics; - private final AtomicLong readCount; - private final AtomicLong offsetOfFirstRead; - private FileType fileType = null; - private String sizeReadByFirstRead; - private String offsetDiffBetweenFirstAndSecondRead; - - /** - * Constructor to initialize the file type metrics. - */ - private FileTypeMetrics() { - collectMetrics = new AtomicBoolean(false); - collectMetricsForNextRead = new AtomicBoolean(false); - collectLenMetrics = new AtomicBoolean(false); - readCount = new AtomicLong(0); - offsetOfFirstRead = new AtomicLong(0); - } - - /** - * Updates the file type based on the metrics collected. - */ - private void updateFileType() { - if (fileType == null) { - fileType = collectMetrics.get() && readCount.get() >= 2 - && haveEqualValues(sizeReadByFirstRead) - && haveEqualValues(offsetDiffBetweenFirstAndSecondRead) ? PARQUET : NON_PARQUET; - } - } - - /** - * Checks if the given value has equal parts. - * - * @param value the value to check - * @return true if the value has equal parts, false otherwise - */ - private boolean haveEqualValues(String value) { - String[] parts = value.split("_"); - return parts.length == 2 - && parts[0].equals(parts[1]); - } - - /** - * Increments the read count. - */ - private void incrementReadCount() { - readCount.incrementAndGet(); - } - - /** - * Returns the read count. - * - * @return the read count - */ - private long getReadCount() { - return readCount.get(); - } - - /** - * Sets the collect metrics flag. - * - * @param collect the value to set - */ - private void setCollectMetrics(boolean collect) { - collectMetrics.set(collect); - } - - /** - * Returns the collect metrics flag. - * - * @return the collect metrics flag - */ - private boolean getCollectMetrics() { - return collectMetrics.get(); - } - - /** - * Sets the collect metrics for the next read flag. - * - * @param collect the value to set - */ - private void setCollectMetricsForNextRead(boolean collect) { - collectMetricsForNextRead.set(collect); - } - - /** - * Returns the collect metrics for the next read flag. - * - * @return the collect metrics for the next read flag - */ - private boolean getCollectMetricsForNextRead() { - return collectMetricsForNextRead.get(); - } - - /** - * Returns the collect length metrics flag. - * - * @return the collect length metrics flag - */ - private boolean getCollectLenMetrics() { - return collectLenMetrics.get(); - } - - /** - * Sets the collect length metrics flag. - * - * @param collect the value to set - */ - private void setCollectLenMetrics(boolean collect) { - collectLenMetrics.set(collect); - } - - /** - * Sets the offset of the first read. - * - * @param offset the value to set - */ - private void setOffsetOfFirstRead(long offset) { - offsetOfFirstRead.set(offset); - } - - /** - * Returns the offset of the first read. - * - * @return the offset of the first read - */ - private long getOffsetOfFirstRead() { - return offsetOfFirstRead.get(); - } - - /** - * Sets the size read by the first read. - * - * @param size the value to set - */ - private void setSizeReadByFirstRead(String size) { - sizeReadByFirstRead = size; - } - - /** - * Returns the size read by the first read. - * - * @return the size read by the first read - */ - private String getSizeReadByFirstRead() { - return sizeReadByFirstRead; - } - - /** - * Sets the offset difference between the first and second read. - * - * @param offsetDiff the value to set - */ - private void setOffsetDiffBetweenFirstAndSecondRead(String offsetDiff) { - offsetDiffBetweenFirstAndSecondRead = offsetDiff; - } - - /** - * Returns the offset difference between the first and second read. - * - * @return the offset difference between the first and second read - */ - private String getOffsetDiffBetweenFirstAndSecondRead() { - return offsetDiffBetweenFirstAndSecondRead; - } - - /** - * Returns the file type. - * - * @return the file type - */ - private FileType getFileType() { - return fileType; - } + private FileTypeMetrics() { + collectMetrics = new AtomicBoolean(false); + collectMetricsForNextRead = new AtomicBoolean(false); + collectLenMetrics = new AtomicBoolean(false); + readCount = new AtomicLong(0); + offsetOfFirstRead = new AtomicLong(0); } /** - * Constructor to initialize the IOStatisticsStore with counters and mean statistics. + * Updates the file type based on the metrics collected. + * In case the first two reads have equal size and offset differences, + * the file is classified as PARQUET; otherwise, it is classified as NON_PARQUET. */ - public AbfsReadFooterMetrics() { - IOStatisticsStore ioStatisticsStore = iostatisticsStore() - .withCounters(getMetricNames(TYPE_COUNTER)) - .withMeanStatistics(getMetricNames(TYPE_MEAN)) - .build(); - setIOStatistics(ioStatisticsStore); + private void updateFileType() { + if (fileType == null) { + fileType = collectMetrics.get() && readCount.get() >= 2 + && haveEqualValues(sizeReadByFirstRead) + && haveEqualValues(offsetDiffBetweenFirstAndSecondRead) + ? PARQUET + : NON_PARQUET; + } } /** - * Returns the metric names for a specific statistic type. + * Checks if the given value has equal parts. * - * @param type the statistic type - * @return the metric names + * @param value the value to check + * @return true if the value has equal parts, false otherwise */ - private String[] getMetricNames(StatisticTypeEnum type) { - return Arrays.stream(AbfsReadFooterMetricsEnum.values()) - .filter(readFooterMetricsEnum -> readFooterMetricsEnum.getStatisticType().equals(type)) - .flatMap(readFooterMetricsEnum -> - FILE.equals(readFooterMetricsEnum.getType()) - ? FILE_TYPE_LIST.stream().map(fileType -> - getMetricName(fileType, readFooterMetricsEnum)) - : Stream.of(readFooterMetricsEnum.getName())) - .toArray(String[]::new); + private boolean haveEqualValues(String value) { + String[] parts = value.split(UNDERSCORE); + return parts.length == 2 + && parts[0].equals(parts[1]); } /** - * Returns the metric name for a specific file type and metric. - * - * @param fileType the type of the file - * @param readFooterMetricsEnum the metric to get the name for - * @return the metric name + * Increments the read count. */ - private String getMetricName(FileType fileType, - AbfsReadFooterMetricsEnum readFooterMetricsEnum) { - if (fileType == null || readFooterMetricsEnum == null) { - LOG.error("File type or ABFS read footer metrics should not be null"); - return EMPTY_STRING; - } - return fileType + COLON + readFooterMetricsEnum.getName(); + private void incrementReadCount() { + readCount.incrementAndGet(); } /** - * Looks up the counter value for a specific metric. + * Returns the read count. * - * @param fileType the type of the file - * @param abfsReadFooterMetricsEnum the metric to look up - * @return the counter value + * @return the read count */ - private long getCounterMetricValue(FileType fileType, - AbfsReadFooterMetricsEnum abfsReadFooterMetricsEnum) { - return lookupCounterValue(getMetricName(fileType, abfsReadFooterMetricsEnum)); + private long getReadCount() { + return readCount.get(); } /** - * Looks up the mean statistic value for a specific metric. + * Sets the collect metrics flag. * - * @param fileType the type of the file - * @param abfsReadFooterMetricsEnum the metric to look up - * @return the mean statistic value + * @param collect the value to set */ - private String getMeanMetricValue(FileType fileType, - AbfsReadFooterMetricsEnum abfsReadFooterMetricsEnum) { - return format(DOUBLE_PRECISION_FORMAT, - lookupMeanStatistic(getMetricName(fileType, abfsReadFooterMetricsEnum))); + private void setCollectMetrics(boolean collect) { + collectMetrics.set(collect); } /** - * Increments the value of a specific metric. + * Returns the collect metrics flag. * - * @param fileType the type of the file - * @param abfsReadFooterMetricsEnum the metric to increment + * @return the collect metrics flag */ - public void incrementMetricValue(FileType fileType, - AbfsReadFooterMetricsEnum abfsReadFooterMetricsEnum) { - incCounterValue(getMetricName(fileType, abfsReadFooterMetricsEnum)); + private boolean getCollectMetrics() { + return collectMetrics.get(); } /** - * Adds a mean statistic value for a specific metric. + * Sets the collect metrics for the next read flag. * - * @param fileType the type of the file - * @param abfsReadFooterMetricsEnum the metric to update - * @param value the new value of the metric + * @param collect the value to set */ - public void addMeanMetricValue(FileType fileType, - AbfsReadFooterMetricsEnum abfsReadFooterMetricsEnum, - long value) { - addMeanStatistic(getMetricName(fileType, abfsReadFooterMetricsEnum), value); + private void setCollectMetricsForNextRead(boolean collect) { + collectMetricsForNextRead.set(collect); } /** - * Returns the total number of files. + * Returns the collect metrics for the next read flag. * - * @return the total number of files + * @return the collect metrics for the next read flag */ - public Long getTotalFiles() { - return getCounterMetricValue(PARQUET, TOTAL_FILES) + getCounterMetricValue(NON_PARQUET, TOTAL_FILES); + private boolean getCollectMetricsForNextRead() { + return collectMetricsForNextRead.get(); } /** - * Updates the map with a new file path identifier. + * Returns the collect length metrics flag. * - * @param filePathIdentifier the file path identifier + * @return the collect length metrics flag */ - public void updateMap(String filePathIdentifier) { - fileTypeMetricsMap.computeIfAbsent(filePathIdentifier, key -> new FileTypeMetrics()); + private boolean getCollectLenMetrics() { + return collectLenMetrics.get(); } /** - * Checks and updates the metrics for a given file read. + * Sets the collect length metrics flag. * - * @param filePathIdentifier the file path identifier - * @param len the length of the read - * @param contentLength the total content length of the file - * @param nextReadPos the position of the next read + * @param collect the value to set */ - public void updateReadMetrics(final String filePathIdentifier, - final int len, - final long contentLength, - final long nextReadPos) { - FileTypeMetrics fileTypeMetrics = fileTypeMetricsMap.computeIfAbsent(filePathIdentifier, key -> new FileTypeMetrics()); - if (fileTypeMetrics.getReadCount() == 0 || (fileTypeMetrics.getReadCount() >= 1 && fileTypeMetrics.getCollectMetrics())) { - updateMetrics(fileTypeMetrics, len, contentLength, nextReadPos); - } + private void setCollectLenMetrics(boolean collect) { + collectLenMetrics.set(collect); } /** - * Updates metrics for a specific file identified by filePathIdentifier. + * Sets the offset of the first read. * - * @param fileTypeMetrics File metadata to know file type. - * @param len The length of the read operation. - * @param contentLength The total content length of the file. - * @param nextReadPos The position of the next read operation. + * @param offset the value to set */ - private void updateMetrics(FileTypeMetrics fileTypeMetrics, - int len, - long contentLength, - long nextReadPos) { - fileTypeMetrics.incrementReadCount(); - - long readCount = fileTypeMetrics.getReadCount(); - - if (readCount == 1) { - handleFirstRead(fileTypeMetrics, nextReadPos, len, contentLength); - } else if (readCount == 2) { - handleSecondRead(fileTypeMetrics, nextReadPos, len, contentLength); - } else { - handleFurtherRead(fileTypeMetrics, len); - } + private void setOffsetOfFirstRead(long offset) { + offsetOfFirstRead.set(offset); } /** - * Handles the first read operation by checking if the current read position is near the end of the file. - * If it is, updates the {@link FileTypeMetrics} object to enable metrics collection and records the first read's - * offset and size. + * Returns the offset of the first read. * - * @param fileTypeMetrics The {@link FileTypeMetrics} object to update with metrics and read details. - * @param nextReadPos The position where the next read will start. - * @param len The length of the current read operation. - * @param contentLength The total length of the file content. + * @return the offset of the first read */ - private void handleFirstRead(FileTypeMetrics fileTypeMetrics, - long nextReadPos, - int len, - long contentLength) { - if (nextReadPos >= contentLength - (long) Integer.parseInt(FOOTER_LENGTH) * ONE_KB) { - fileTypeMetrics.setCollectMetrics(true); - fileTypeMetrics.setCollectMetricsForNextRead(true); - fileTypeMetrics.setOffsetOfFirstRead(nextReadPos); - fileTypeMetrics.setSizeReadByFirstRead(len + "_" + Math.abs(contentLength - nextReadPos)); - } + private long getOffsetOfFirstRead() { + return offsetOfFirstRead.get(); } /** - * Handles the second read operation by checking if metrics collection is enabled for the next read. - * If it is, calculates the offset difference between the first and second reads, updates the {@link FileTypeMetrics} - * object with this information, and sets the file type. Then, updates the metrics data. + * Sets the size read by the first read. * - * @param fileTypeMetrics The {@link FileTypeMetrics} object to update with metrics and read details. - * @param nextReadPos The position where the next read will start. - * @param len The length of the current read operation. - * @param contentLength The total length of the file content. + * @param size the value to set */ - private void handleSecondRead(FileTypeMetrics fileTypeMetrics, - long nextReadPos, - int len, - long contentLength) { - if (fileTypeMetrics.getCollectMetricsForNextRead()) { - long offsetDiff = Math.abs(nextReadPos - fileTypeMetrics.getOffsetOfFirstRead()); - fileTypeMetrics.setOffsetDiffBetweenFirstAndSecondRead(len + "_" + offsetDiff); - fileTypeMetrics.setCollectLenMetrics(true); - fileTypeMetrics.updateFileType(); - updateMetricsData(fileTypeMetrics, len, contentLength); - } + private void setSizeReadByFirstRead(String size) { + sizeReadByFirstRead = size; } /** - * Handles further read operations beyond the second read. If metrics collection is enabled and the file type is set, - * updates the read length requested and increments the read count for the specific file type. + * Returns the size read by the first read. * - * @param fileTypeMetrics The {@link FileTypeMetrics} object containing metrics and read details. - * @param len The length of the current read operation. + * @return the size read by the first read */ - private void handleFurtherRead(FileTypeMetrics fileTypeMetrics, int len) { - if (fileTypeMetrics.getCollectLenMetrics() && fileTypeMetrics.getFileType() != null) { - FileType fileType = fileTypeMetrics.getFileType(); - addMeanMetricValue(fileType, AVG_READ_LEN_REQUESTED, len); - } + private String getSizeReadByFirstRead() { + return sizeReadByFirstRead; } /** - * Updates the metrics data for a specific file identified by the {@link FileTypeMetrics} object. - * This method calculates and updates various metrics such as read length requested, file length, - * size read by the first read, and offset differences between reads. + * Sets the offset difference between the first and second read. * - * @param fileTypeMetrics The {@link FileTypeMetrics} object containing metrics and read details. - * @param len The length of the current read operation. - * @param contentLength The total length of the file content. + * @param offsetDiff the value to set */ - private void updateMetricsData(FileTypeMetrics fileTypeMetrics, - int len, - long contentLength) { - long sizeReadByFirstRead = Long.parseLong(fileTypeMetrics.getSizeReadByFirstRead().split("_")[0]); - long firstOffsetDiff = Long.parseLong(fileTypeMetrics.getSizeReadByFirstRead().split("_")[1]); - long secondOffsetDiff = Long.parseLong(fileTypeMetrics.getOffsetDiffBetweenFirstAndSecondRead().split("_")[1]); - FileType fileType = fileTypeMetrics.getFileType(); - - addMeanMetricValue(fileType, AVG_READ_LEN_REQUESTED, len); - addMeanMetricValue(fileType, AVG_READ_LEN_REQUESTED, sizeReadByFirstRead); - addMeanMetricValue(fileType, AVG_FILE_LENGTH, contentLength); - addMeanMetricValue(fileType, AVG_SIZE_READ_BY_FIRST_READ, sizeReadByFirstRead); - addMeanMetricValue(fileType, AVG_OFFSET_DIFF_BETWEEN_FIRST_AND_SECOND_READ, len); - addMeanMetricValue(fileType, AVG_FIRST_OFFSET_DIFF, firstOffsetDiff); - addMeanMetricValue(fileType, AVG_SECOND_OFFSET_DIFF, secondOffsetDiff); - incrementMetricValue(fileType, TOTAL_FILES); + private void setOffsetDiffBetweenFirstAndSecondRead(String offsetDiff) { + offsetDiffBetweenFirstAndSecondRead = offsetDiff; } /** - * Appends the metrics for a specific file type to the given metric builder. + * Returns the offset difference between the first and second read. * - * @param metricBuilder the metric builder to append the metrics to - * @param fileType the file type to append the metrics for + * @return the offset difference between the first and second read */ - private void appendMetrics(StringBuilder metricBuilder, FileType fileType) { - long totalFiles = getCounterMetricValue(fileType, TOTAL_FILES); - if (totalFiles <= 0) { - return; - } - - String sizeReadByFirstRead = getMeanMetricValue(fileType, AVG_SIZE_READ_BY_FIRST_READ); - String offsetDiffBetweenFirstAndSecondRead = getMeanMetricValue(fileType, AVG_OFFSET_DIFF_BETWEEN_FIRST_AND_SECOND_READ); - - if (NON_PARQUET.equals(fileType)) { - sizeReadByFirstRead += CHAR_UNDERSCORE + getMeanMetricValue(fileType, AVG_FIRST_OFFSET_DIFF); - offsetDiffBetweenFirstAndSecondRead += CHAR_UNDERSCORE + getMeanMetricValue(fileType, AVG_SECOND_OFFSET_DIFF); - } - - metricBuilder.append(CHAR_DOLLAR) - .append(fileType) - .append(FIRST_READ) - .append(sizeReadByFirstRead) - .append(SECOND_READ) - .append(offsetDiffBetweenFirstAndSecondRead) - .append(FILE_LENGTH) - .append(getMeanMetricValue(fileType, AVG_FILE_LENGTH)) - .append(READ_LENGTH) - .append(getMeanMetricValue(fileType, AVG_READ_LEN_REQUESTED)); + private String getOffsetDiffBetweenFirstAndSecondRead() { + return offsetDiffBetweenFirstAndSecondRead; } /** - * Returns the read footer metrics for all file types. + * Returns the file type. * - * @return the read footer metrics as a string + * @return the file type */ - @Override - public String toString() { - StringBuilder readFooterMetric = new StringBuilder(); - appendMetrics(readFooterMetric, PARQUET); - appendMetrics(readFooterMetric, NON_PARQUET); - return readFooterMetric.toString(); + private FileType getFileType() { + return fileType; + } + } + + /** + * Constructor to initialize the IOStatisticsStore with counters and mean statistics. + */ + public AbfsReadFooterMetrics() { + IOStatisticsStore ioStatisticsStore = iostatisticsStore() + .withCounters(getMetricNames(TYPE_COUNTER)) + .withMeanStatistics(getMetricNames(TYPE_MEAN)) + .build(); + setIOStatistics(ioStatisticsStore); + } + + /** + * Constructor to initialize the IOStatisticsStore with counters and mean statistics, + * and a map to track file type metrics. + * + * @param fileTypeMetricsMap the map to track file type metrics + */ + public AbfsReadFooterMetrics(Map fileTypeMetricsMap) { + this(); + this.fileTypeMetricsMap = fileTypeMetricsMap == null + ? new ConcurrentHashMap<>() + : fileTypeMetricsMap; + } + + /** + * Returns the map of file type metrics. + * + * @return the map of file type metrics + */ + public Map getFileTypeMetricsMap() { + return fileTypeMetricsMap; + } + + /** + * Returns the metric names for a specific statistic type. + * + * @param type the statistic type + * @return the metric names + */ + private String[] getMetricNames(StatisticTypeEnum type) { + return Arrays.stream(AbfsReadFooterMetricsEnum.values()) + .filter( + readFooterMetricsEnum -> readFooterMetricsEnum.getStatisticType() + .equals(type)) + .flatMap(readFooterMetricsEnum -> + FILE.equals(readFooterMetricsEnum.getType()) + ? FILE_TYPE_LIST.stream().map(fileType -> + getMetricName(fileType, readFooterMetricsEnum)) + : Stream.of(readFooterMetricsEnum.getName())) + .toArray(String[]::new); + } + + /** + * Returns the metric name for a specific file type and metric. + * + * @param fileType the type of the file + * @param readFooterMetricsEnum the metric to get the name for + * @return the metric name + */ + private String getMetricName(FileType fileType, + AbfsReadFooterMetricsEnum readFooterMetricsEnum) { + if (fileType == null || readFooterMetricsEnum == null) { + LOG.error("File type or ABFS read footer metrics should not be null"); + return EMPTY_STRING; + } + return fileType + COLON + readFooterMetricsEnum.getName(); + } + + /** + * Looks up the counter value for a specific metric. + * + * @param fileType the type of the file + * @param abfsReadFooterMetricsEnum the metric to look up + * @return the counter value + */ + private long getCounterMetricValue(FileType fileType, + AbfsReadFooterMetricsEnum abfsReadFooterMetricsEnum) { + return lookupCounterValue( + getMetricName(fileType, abfsReadFooterMetricsEnum)); + } + + /** + * Looks up the mean statistic value for a specific metric. + * + * @param fileType the type of the file + * @param abfsReadFooterMetricsEnum the metric to look up + * @return the mean statistic value + */ + private String getMeanMetricValue(FileType fileType, + AbfsReadFooterMetricsEnum abfsReadFooterMetricsEnum) { + return format(DOUBLE_PRECISION_FORMAT, + lookupMeanStatistic( + getMetricName(fileType, abfsReadFooterMetricsEnum))); + } + + /** + * Increments the value of a specific metric. + * + * @param fileType the type of the file + * @param abfsReadFooterMetricsEnum the metric to increment + */ + public void incrementMetricValue(FileType fileType, + AbfsReadFooterMetricsEnum abfsReadFooterMetricsEnum) { + incCounterValue(getMetricName(fileType, abfsReadFooterMetricsEnum)); + } + + /** + * Adds a mean statistic value for a specific metric. + * + * @param fileType the type of the file + * @param abfsReadFooterMetricsEnum the metric to update + * @param value the new value of the metric + */ + public void addMeanMetricValue(FileType fileType, + AbfsReadFooterMetricsEnum abfsReadFooterMetricsEnum, + long value) { + addMeanStatistic(getMetricName(fileType, abfsReadFooterMetricsEnum), value); + } + + /** + * Returns the total number of files. + * + * @return the total number of files + */ + public Long getTotalFiles() { + return getCounterMetricValue(PARQUET, TOTAL_FILES) + getCounterMetricValue( + NON_PARQUET, TOTAL_FILES); + } + + /** + * Updates the map with a new file path identifier. + * + * @param filePathIdentifier the file path identifier + */ + public void updateMap(String filePathIdentifier) { + fileTypeMetricsMap.computeIfAbsent(filePathIdentifier, + key -> new FileTypeMetrics()); + } + + /** + * Checks and updates the metrics for a given file read. + * + * @param filePathIdentifier the file path identifier + * @param len the length of the read + * @param contentLength the total content length of the file + * @param nextReadPos the position of the next read + */ + public void updateReadMetrics(final String filePathIdentifier, + final int len, + final long contentLength, + final long nextReadPos) { + FileTypeMetrics fileTypeMetrics = fileTypeMetricsMap.computeIfAbsent( + filePathIdentifier, key -> new FileTypeMetrics()); + if (fileTypeMetrics.getReadCount() == 0 || ( + fileTypeMetrics.getReadCount() >= 1 + && fileTypeMetrics.getCollectMetrics())) { + updateMetrics(fileTypeMetrics, len, contentLength, nextReadPos); + } + } + + /** + * Updates metrics for a specific file identified by filePathIdentifier. + * + * @param fileTypeMetrics File metadata to know file type. + * @param len The length of the read operation. + * @param contentLength The total content length of the file. + * @param nextReadPos The position of the next read operation. + */ + private void updateMetrics(FileTypeMetrics fileTypeMetrics, + int len, + long contentLength, + long nextReadPos) { + fileTypeMetrics.incrementReadCount(); + + long readCount = fileTypeMetrics.getReadCount(); + + if (readCount == 1) { + handleFirstRead(fileTypeMetrics, nextReadPos, len, contentLength); + } else if (readCount == 2) { + handleSecondRead(fileTypeMetrics, nextReadPos, len, contentLength); + } else { + handleFurtherRead(fileTypeMetrics, len); + } + } + + /** + * Handles the first read operation by checking if the current read position is near the end of the file. + * If it is, updates the {@link FileTypeMetrics} object to enable metrics collection and records the first read's + * offset and size. + * + * @param fileTypeMetrics The {@link FileTypeMetrics} object to update with metrics and read details. + * @param nextReadPos The position where the next read will start. + * @param len The length of the current read operation. + * @param contentLength The total length of the file content. + */ + private void handleFirstRead(FileTypeMetrics fileTypeMetrics, + long nextReadPos, + int len, + long contentLength) { + if (nextReadPos + >= contentLength - (long) Integer.parseInt(FOOTER_LENGTH) * ONE_KB) { + fileTypeMetrics.setCollectMetrics(true); + fileTypeMetrics.setCollectMetricsForNextRead(true); + fileTypeMetrics.setOffsetOfFirstRead(nextReadPos); + fileTypeMetrics.setSizeReadByFirstRead( + len + UNDERSCORE + Math.abs(contentLength - nextReadPos)); + } + } + + /** + * Handles the second read operation by checking if metrics collection is enabled for the next read. + * If it is, calculates the offset difference between the first and second reads, updates the {@link FileTypeMetrics} + * object with this information, and sets the file type. Then, updates the metrics data. + * + * @param fileTypeMetrics The {@link FileTypeMetrics} object to update with metrics and read details. + * @param nextReadPos The position where the next read will start. + * @param len The length of the current read operation. + * @param contentLength The total length of the file content. + */ + private void handleSecondRead(FileTypeMetrics fileTypeMetrics, + long nextReadPos, + int len, + long contentLength) { + if (fileTypeMetrics.getCollectMetricsForNextRead()) { + long offsetDiff = Math.abs( + nextReadPos - fileTypeMetrics.getOffsetOfFirstRead()); + fileTypeMetrics.setOffsetDiffBetweenFirstAndSecondRead( + len + UNDERSCORE + offsetDiff); + fileTypeMetrics.setCollectLenMetrics(true); + fileTypeMetrics.updateFileType(); + updateMetricsData(fileTypeMetrics, len, contentLength); } + } + + /** + * Handles further read operations beyond the second read. If metrics collection is enabled and the file type is set, + * updates the read length requested and increments the read count for the specific file type. + * + * @param fileTypeMetrics The {@link FileTypeMetrics} object containing metrics and read details. + * @param len The length of the current read operation. + */ + private void handleFurtherRead(FileTypeMetrics fileTypeMetrics, int len) { + if (fileTypeMetrics.getCollectLenMetrics() + && fileTypeMetrics.getFileType() != null) { + FileType fileType = fileTypeMetrics.getFileType(); + addMeanMetricValue(fileType, AVG_READ_LEN_REQUESTED, len); + } + } + + /** + * Updates the metrics data for a specific file identified by the {@link FileTypeMetrics} object. + * This method calculates and updates various metrics such as read length requested, file length, + * size read by the first read, and offset differences between reads. + * + * @param fileTypeMetrics The {@link FileTypeMetrics} object containing metrics and read details. + * @param len The length of the current read operation. + * @param contentLength The total length of the file content. + */ + private void updateMetricsData(FileTypeMetrics fileTypeMetrics, + int len, + long contentLength) { + long sizeReadByFirstRead = Long.parseLong( + fileTypeMetrics.getSizeReadByFirstRead().split(UNDERSCORE)[0]); + long firstOffsetDiff = Long.parseLong( + fileTypeMetrics.getSizeReadByFirstRead().split(UNDERSCORE)[1]); + long secondOffsetDiff = Long.parseLong( + fileTypeMetrics.getOffsetDiffBetweenFirstAndSecondRead().split( + UNDERSCORE)[1]); + FileType fileType = fileTypeMetrics.getFileType(); + + addMeanMetricValue(fileType, AVG_READ_LEN_REQUESTED, len); + addMeanMetricValue(fileType, AVG_READ_LEN_REQUESTED, sizeReadByFirstRead); + addMeanMetricValue(fileType, AVG_FILE_LENGTH, contentLength); + addMeanMetricValue(fileType, AVG_SIZE_READ_BY_FIRST_READ, + sizeReadByFirstRead); + addMeanMetricValue(fileType, AVG_OFFSET_DIFF_BETWEEN_FIRST_AND_SECOND_READ, + len); + addMeanMetricValue(fileType, AVG_FIRST_OFFSET_DIFF, firstOffsetDiff); + addMeanMetricValue(fileType, AVG_SECOND_OFFSET_DIFF, secondOffsetDiff); + incrementMetricValue(fileType, TOTAL_FILES); + } + + /** + * Appends the metrics for a specific file type to the given metric builder. + * + * @param metricBuilder the metric builder to append the metrics to + * @param fileType the file type to append the metrics for + */ + private void appendMetrics(StringBuilder metricBuilder, FileType fileType) { + long totalFiles = getCounterMetricValue(fileType, TOTAL_FILES); + if (totalFiles <= 0) { + return; + } + + String sizeReadByFirstRead = getMeanMetricValue(fileType, + AVG_SIZE_READ_BY_FIRST_READ); + String offsetDiffBetweenFirstAndSecondRead = getMeanMetricValue(fileType, + AVG_OFFSET_DIFF_BETWEEN_FIRST_AND_SECOND_READ); + + if (NON_PARQUET.equals(fileType)) { + sizeReadByFirstRead += CHAR_UNDERSCORE + getMeanMetricValue(fileType, + AVG_FIRST_OFFSET_DIFF); + offsetDiffBetweenFirstAndSecondRead += CHAR_UNDERSCORE + + getMeanMetricValue(fileType, AVG_SECOND_OFFSET_DIFF); + } + + appendAlways(metricBuilder, CHAR_DOLLAR, fileType.toString()); + appendAlways(metricBuilder, FIRST_READ, sizeReadByFirstRead); + appendAlways(metricBuilder, SECOND_READ, + offsetDiffBetweenFirstAndSecondRead); + appendAlways(metricBuilder, FILE_LENGTH, + getMeanMetricValue(fileType, AVG_FILE_LENGTH)); + appendAlways(metricBuilder, READ_LENGTH, + getMeanMetricValue(fileType, AVG_READ_LEN_REQUESTED)); + } + + /** + * Returns the read footer metrics for all file types. + * + * @return the read footer metrics as a string + */ + @Override + public String toString() { + StringBuilder readFooterMetric = new StringBuilder(); + appendMetrics(readFooterMetric, PARQUET); + appendMetrics(readFooterMetric, NON_PARQUET); + return readFooterMetric.toString(); + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java index ff7300d280fdb..1b55084fb4571 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java @@ -110,7 +110,7 @@ public class AbfsRestOperation { private final int maxIoRetries; private AbfsHttpOperation result; private final AbfsCounters abfsCounters; - private AbfsBackoffMetrics abfsBackoffMetrics; + private final AbfsBackoffMetrics abfsBackoffMetrics; /** * This variable contains the reason of last API call within the same * AbfsRestOperation object. @@ -239,6 +239,8 @@ String getSasToken() { this.abfsCounters = client.getAbfsCounters(); if (abfsCounters != null) { this.abfsBackoffMetrics = abfsCounters.getAbfsBackoffMetrics(); + } else { + this.abfsBackoffMetrics = null; } this.maxIoRetries = abfsConfiguration.getMaxIoRetries(); this.intercept = client.getIntercept(); @@ -293,7 +295,10 @@ public void execute(TracingContext tracingContext) if (abfsCounters != null) { abfsCounters.getLastExecutionTime().set(now()); } - client.timerOrchestrator(TimerFunctionality.RESUME, null); + if (client.getAbfsMetricsManager() != null) { + client.getAbfsMetricsManager() + .timerOrchestrator(TimerFunctionality.RESUME, null); + } IOStatisticsBinding.trackDurationOfInvocation(abfsCounters, AbfsStatistic.getStatNameFromHttpCall(method), () -> completeExecute(lastUsedTracingContext)); @@ -324,8 +329,7 @@ void completeExecute(TracingContext tracingContext) retryCount = 0; retryPolicy = client.getExponentialRetryPolicy(); LOG.debug("First execution of REST operation - {}", operationType); - long sleepDuration = 0L; - if (abfsBackoffMetrics != null) { + if (abfsBackoffMetrics != null && !tracingContext.isMetricCall()) { synchronized (this) { abfsBackoffMetrics.incrementMetricValue(TOTAL_NUMBER_OF_REQUESTS); } @@ -338,7 +342,7 @@ void completeExecute(TracingContext tracingContext) LOG.debug("Rest operation {} failed with failureReason: {}. Retrying with retryCount = {}, retryPolicy: {} and sleepInterval: {}", operationType, failureReason, retryCount, retryPolicy.getAbbreviation(), retryInterval); if (abfsBackoffMetrics != null) { - updateBackoffTimeMetrics(retryCount, sleepDuration); + updateBackoffTimeMetrics(retryCount, retryInterval); } Thread.sleep(retryInterval); } catch (InterruptedException ex) { @@ -412,7 +416,7 @@ private boolean executeHttpOperation(final int retryCount, incrementCounter(AbfsStatistic.CONNECTIONS_MADE, 1); tracingContext.constructHeader(httpOperation, failureReason, retryPolicy.getAbbreviation()); - signRequest(httpOperation, hasRequestBody ? bufferLength : 0); + signRequest(httpOperation, hasRequestBody ? bufferLength : 0, tracingContext.isMetricCall()); } catch (IOException e) { LOG.debug("Auth failure: {}, {}", method, url); @@ -580,9 +584,12 @@ private void registerApacheHttpClientIoException() { * @throws IOException failure */ @VisibleForTesting - public void signRequest(final AbfsHttpOperation httpOperation, int bytesToSign) throws IOException { - if (client.isSendMetricCall()) { - client.getMetricSharedkeyCredentials().signRequest(httpOperation, bytesToSign); + public void signRequest(final AbfsHttpOperation httpOperation, int bytesToSign, + boolean isMetricCall) throws IOException { + if (isMetricCall && client.getAbfsMetricsManager() != null + && client.getAbfsMetricsManager().hasSeparateMetricAccount()) { + client.getAbfsMetricsManager().getMetricSharedkeyCredentials() + .signRequest(httpOperation, bytesToSign); } else { switch (client.getAuthType()) { case Custom: diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbstractAbfsStatisticsSource.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbstractAbfsStatisticsSource.java index a8f69cf72e2ce..3eeb73b70964c 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbstractAbfsStatisticsSource.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbstractAbfsStatisticsSource.java @@ -141,6 +141,30 @@ protected double lookupMeanStatistic(String name) { return ioStatisticsStore.meanStatistics().get(name).mean(); } + /** + * Appends the label and value to the StringBuilder if the value is positive. + * + * @param sb the StringBuilder to append to + * @param label the label to append + * @param value the value to check and append + */ + protected void appendIfPositive(StringBuilder sb, String label, long value) { + if (value > 0) { + sb.append(label).append(value); + } + } + + /** + * Appends the label and value to the StringBuilder unconditionally. + * + * @param sb the StringBuilder to append to + * @param label the label to append + * @param value the value to append + */ + protected void appendAlways(StringBuilder sb, String label, String value) { + sb.append(label).append(value); + } + /** * Returns a string representation of the AbstractAbfsStatisticsSource. * diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AggregateMetricsManager.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AggregateMetricsManager.java new file mode 100644 index 0000000000000..e8bf3c0e8138e --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AggregateMetricsManager.java @@ -0,0 +1,176 @@ +/** + * 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.hadoop.fs.azurebfs.services; + +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; +import org.apache.hadoop.fs.azurebfs.utils.SimpleRateLimiter; + +/** + * AggregateMetricsManager manages metrics collection and dispatching + * for multiple AbfsClients across different accounts. + */ +public final class AggregateMetricsManager { + + // Singleton instance of AggregateMetricsManager. + private static volatile AggregateMetricsManager instance; + + // Rate limiter to control the rate of dispatching metrics. + private static volatile SimpleRateLimiter rateLimiter; + + // Map of account name to MetricsBucket. + private final ConcurrentHashMap buckets = + new ConcurrentHashMap<>(); + + // Scheduler for periodic dispatching of metrics. + private final ScheduledExecutorService scheduler; + + // Private constructor to enforce singleton pattern. + private AggregateMetricsManager(final long dispatchIntervalInMins, + final int permitsPerSecond) throws InvalidConfigurationValueException { + + if (dispatchIntervalInMins <= 0) { + throw new InvalidConfigurationValueException( + "dispatchIntervalInMins must be > 0"); + } + + if (permitsPerSecond <= 0) { + throw new InvalidConfigurationValueException( + "permitsPerSecond must be > 0"); + } + + rateLimiter = new SimpleRateLimiter(permitsPerSecond); + + // Initialize scheduler for periodic dispatching of metrics. + this.scheduler = Executors.newSingleThreadScheduledExecutor(r -> { + Thread t = new Thread(r, "ABFS-Aggregated-Metrics-Dispatcher"); + t.setDaemon(true); + return t; + }); + + // Schedule periodic dispatching of metrics. + this.scheduler.scheduleWithFixedDelay( + this::dispatchMetrics, + dispatchIntervalInMins, + dispatchIntervalInMins, + TimeUnit.MINUTES); + + // Add shutdown hook to dispatch remaining metrics on JVM shutdown. + Runtime.getRuntime().addShutdownHook(new Thread(() -> { + dispatchMetrics(); + scheduler.shutdown(); + })); + } + + /** + * Get the singleton instance of AggregateMetricsManager. + * + * @param dispatchIntervalInMins Interval in minutes for dispatching metrics. + * @param permitsPerSecond Rate limit for dispatching metrics. + * @return Singleton instance of AggregateMetricsManager. + */ + public static AggregateMetricsManager getInstance(final long dispatchIntervalInMins, + final int permitsPerSecond) { + if (instance != null) { + return instance; + } + + synchronized (AggregateMetricsManager.class) { + if (instance == null) { + try { + instance = new AggregateMetricsManager( + dispatchIntervalInMins, permitsPerSecond); + } catch (InvalidConfigurationValueException e) { + throw new RuntimeException( + "Failed to initialize AggregateMetricsManager", e); + } + } + return instance; + } + } + + /** + * Register an AbfsClient with the manager. + * @param account Account name. + * @param abfsClient AbfsClient instance. + */ + public void registerClient(String account, AbfsClient abfsClient) { + if (StringUtils.isEmpty(account) || abfsClient == null) { + return; + } + + buckets.computeIfAbsent(account, + key -> new MetricsBucket(rateLimiter)) + .registerClient(abfsClient); + } + + /** + * Deregister an AbfsClient from the manager. + * @param account Account name. + * @param abfsClient AbfsClient instance. + * @return true if the client was deregistered, false otherwise. + */ + public boolean deregisterClient(String account, AbfsClient abfsClient) { + if (StringUtils.isEmpty(account) || abfsClient == null) { + return false; + } + + AtomicBoolean isRemoved = new AtomicBoolean(false); + + buckets.computeIfPresent(account, (key, bucket) -> { + // Deregister the client + isRemoved.set(bucket.deregisterClient(abfsClient)); + + // If bucket became empty, remove it atomically + return bucket.isEmpty() ? null : bucket; + }); + + return isRemoved.get(); + } + + /** + * Record metrics data for a specific account. + * @param accountName Account name. + * @param metricsData Metrics data to record. + */ + public void recordMetric(String accountName, String metricsData) { + if (StringUtils.isEmpty(accountName) + || StringUtils.isEmpty(metricsData)) { + return; + } + + MetricsBucket bucket = buckets.get(accountName); + if (bucket == null) { + return; + } + + bucket.addRequest(metricsData); + } + + // Dispatch metrics for all buckets. + private void dispatchMetrics() { + buckets.values().forEach(MetricsBucket::drainAndSendIfReady); + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/MetricsBucket.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/MetricsBucket.java new file mode 100644 index 0000000000000..47f72c581cc5e --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/MetricsBucket.java @@ -0,0 +1,209 @@ +/** + * 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.hadoop.fs.azurebfs.services; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.azurebfs.utils.SimpleRateLimiter; + +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CLOSING_SQUARE_BRACKET; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COLON; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.OPENING_SQUARE_BRACKET; + +/** + * MetricsBucket holds metrics for multiple AbfsClients and + * dispatches them in batches, respecting rate limits. + */ +final class MetricsBucket { + + // Logger for the class. + private static final Logger LOG = LoggerFactory.getLogger(MetricsBucket.class); + + // Rate limiter to control the rate of dispatching metrics. + private final SimpleRateLimiter rateLimiter; + + // Buffer to hold metrics before sending. + private final AtomicReference> metricsBuffer = + new AtomicReference<>(new ConcurrentLinkedQueue<>()); + + // Set of registered AbfsClients. + private final Set clients = + ConcurrentHashMap.newKeySet(); + + // Maximum size of metrics header in characters. + private static final long MAX_HEADER_SIZE = 1024; + + // Constructor + MetricsBucket(SimpleRateLimiter rateLimiter) { + this.rateLimiter = rateLimiter; + } + + /** + * Register a new AbfsClient. + * @param client the AbfsClient to register + */ + public void registerClient(AbfsClient client) { + if (client != null) { + clients.add(client); + } + } + + /** + * Deregister an AbfsClient. If this is the last client, drain and send + * any remaining metrics. + * @param client the AbfsClient to deregister + * @return true if the client was deregistered, false otherwise + */ + public boolean deregisterClient(AbfsClient client) { + if (client == null) { + return false; + } + ConcurrentLinkedQueue batchToSend = null; + boolean isLastClient = false; + + synchronized (this) { + if (!clients.contains(client)) { + return false; + } + + if (clients.size() == 1) { + // This client is the last one — drain metrics now + batchToSend = metricsBuffer.getAndSet(new ConcurrentLinkedQueue<>()); + isLastClient = true; + } + + clients.remove(client); + } + if (isLastClient) { + sendMetrics(client, batchToSend); + } + return true; + } + + /** + * Add a metric to the buffer. + * @param metric the metric to add + */ + void addRequest(String metric) { + if (metric != null) { + metricsBuffer.get().add(metric); + } + } + + /** + * Drain the metrics buffer and send if there are registered clients. + */ + public void drainAndSendIfReady() { + AbfsClient client; + + synchronized (this) { + if (clients.isEmpty()) { + return; + } + client = clients.iterator().next(); + } + + ConcurrentLinkedQueue batch = metricsBuffer.getAndSet( + new ConcurrentLinkedQueue<>()); + if (batch.isEmpty()) { + return; + } + + sendMetrics(client, batch); + } + + // Send metrics outside synchronized block + private void sendMetrics(AbfsClient client, + ConcurrentLinkedQueue batchToSend) { + // Send outside synchronized block + if (client != null && batchToSend != null && !batchToSend.isEmpty()) { + for (String chunk : splitListBySize(batchToSend, MAX_HEADER_SIZE)) { + rateLimiter.acquire(5, TimeUnit.SECONDS); // Rate limiting + try { + client.getMetricCall(chunk); + } catch (IOException ignored) { + LOG.debug("Failed to send metrics: {}", ignored.getMessage()); + } + } + } + } + + // Check if there are no registered clients + public synchronized boolean isEmpty() { + return clients.isEmpty(); + } + + /** + * Split the list of metrics into chunks that fit within maxChars. + * Each metric is wrapped in square brackets and separated by colons. + */ + private static List splitListBySize( + ConcurrentLinkedQueue items, long maxChars) { + + if (items.isEmpty()) { + return Collections.emptyList(); + } + + List result = new ArrayList<>(); + StringBuilder sb = new StringBuilder(); + + for (String s : items) { + String wrapped = OPENING_SQUARE_BRACKET + s + CLOSING_SQUARE_BRACKET; + int additional = + sb.length() == 0 ? wrapped.length() + : wrapped.length() + 1; + + if (wrapped.length() > maxChars) { + if (sb.length() > 0) { + result.add(sb.toString()); + sb.setLength(0); + } + result.add(wrapped); + continue; + } + + if (sb.length() + additional > maxChars) { + result.add(sb.toString()); + sb.setLength(0); + } + + if (sb.length() > 0) { + sb.append(COLON); + } + sb.append(wrapped); + } + + if (sb.length() > 0) { + result.add(sb.toString()); + } + + return result; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/MetricFormat.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/MetricFormat.java index 48c216ff6e5bb..fdb57e5be20ad 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/MetricFormat.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/MetricFormat.java @@ -15,6 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.hadoop.fs.azurebfs.utils; public enum MetricFormat { @@ -24,13 +25,11 @@ public enum MetricFormat { INTERNAL_FOOTER_METRIC_FORMAT, // :: // : - INTERNAL_METRIC_FORMAT, // :: + INTERNAL_METRIC_FORMAT; // :: // :: - EMPTY; - @Override public String toString() { - return this == EMPTY ? "" : this.name(); + return this.name(); } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/SimpleRateLimiter.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/SimpleRateLimiter.java new file mode 100644 index 0000000000000..c8d5744d59744 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/SimpleRateLimiter.java @@ -0,0 +1,95 @@ +/** + * 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.hadoop.fs.azurebfs.utils; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.LockSupport; + +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; + +/** + * A simple rate limiter that allows a specified number of permits + * per second. This implementation uses basic synchronization and + * LockSupport for waiting. + */ +public final class SimpleRateLimiter { + + // Interval between permits in nanoseconds. + private final long intervalNanos; + + // Next allowed time to acquire a permit in nanoseconds. + private long nextAllowedTime; + + /** Number of nanoseconds in one second. */ + private static final long NANOS_PER_SECOND = 1_000_000_000L; + + /** + * Constructs a SimpleRateLimiter that allows the specified number of + * permits per second. + * + * @param permitsPerSecond Number of permits allowed per second. + * @throws InvalidConfigurationValueException if permitsPerSecond is + * less than or equal to zero. + */ + public SimpleRateLimiter(int permitsPerSecond) + throws InvalidConfigurationValueException { + if (permitsPerSecond <= 0) { + throw new InvalidConfigurationValueException( + "permitsPerSecond must be > 0"); + } + this.intervalNanos = NANOS_PER_SECOND / permitsPerSecond; + this.nextAllowedTime = System.nanoTime(); + } + + /** + * Acquires a permit from the rate limiter, waiting up to the + * specified timeout if necessary. + * + * @param timeout Maximum time to wait for a permit. + * @param unit Time unit of the timeout argument. + */ + public synchronized void acquire(long timeout, TimeUnit unit) { + if (timeout <= 0) { + return; + } + + final long deadline = System.nanoTime() + unit.toNanos(timeout); + while (true) { + long now = System.nanoTime(); + long wait = nextAllowedTime - now; + + if (wait <= 0) { + nextAllowedTime = now + intervalNanos; + return; + } + + long remaining = deadline - now; + if (remaining <= 0) { + return; // timeout expired + } + + LockSupport.parkNanos(Math.min(wait, remaining)); + + if (Thread.interrupted()) { + Thread.currentThread().interrupt(); + return; + } + } + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java index ff3ab405959af..d6dbae2c786d8 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java @@ -198,6 +198,10 @@ public void setListener(Listener listener) { this.listener = listener; } + public boolean isMetricCall() { + return TracingHeaderFormat.AGGREGATED_METRICS_FORMAT.equals(format); + } + /** * Concatenate all components separated by (:) into a string and set into * X_MS_CLIENT_REQUEST_ID header of the http operation @@ -242,13 +246,17 @@ public void constructHeader(AbfsHttpOperation httpOperation, String previousFail + operatedBlobCount + COLON + getOperationSpecificHeader(opType) + COLON + httpOperation.getTracingContextSuffix() + COLON - + metricResults + COLON + resourceUtilizationMetricResults + COLON + + resourceUtilizationMetricResults + COLON + (fnsEndpointConverted ? fnsEndptConvertedIndicator : EMPTY_STRING); break; case TWO_ID_FORMAT: header = TracingHeaderVersion.getCurrentVersion() + COLON + clientCorrelationID + COLON + clientRequestId; break; + case AGGREGATED_METRICS_FORMAT: + header = TracingHeaderVersion.getMetricsCurrentVersion() + COLON + + metricResults; + break; default: //case SINGLE_ID_FORMAT header = TracingHeaderVersion.getCurrentVersion() + COLON diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderFormat.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderFormat.java index 3f23ae3ed7c14..a8a22fcd0c390 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderFormat.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderFormat.java @@ -23,6 +23,8 @@ public enum TracingHeaderFormat { TWO_ID_FORMAT, // : - ALL_ID_FORMAT; // :: + ALL_ID_FORMAT, // :: // :::: + + AGGREGATED_METRICS_FORMAT; // :: } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderVersion.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderVersion.java index b4b3fe4c072a8..c542e42827ca2 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderVersion.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderVersion.java @@ -45,9 +45,18 @@ public enum TracingHeaderVersion { * Schema: version:clientCorrelationId:clientRequestId:fileSystemId * :primaryRequestId:streamId:opType:retryHeader:ingressHandler * :position:operatedBlobCount:operationSpecificHeader:httpOperationHeader - * :aggregatedMetrics:resourceUtilizationMetrics:fnsEndptConvertedIndicator + * :resourceUtilizationMetrics:fnsEndptConvertedIndicator */ - V2("v2", 16); + V2("v2", 15), + /** + * Version 0 of the aggregated metrics tracing header, which includes + * a version prefix and has 3 permanent fields. + * This version is used for the aggregated metrics tracing header schema. + * Schema: metricsVersion:List + * where AggregatedMetrics = clientCorrelationId:fileSystemId:aggregated-metrics + * and AggregatedMetrics is enclosed within [] and separated by : + */ + AV0("av0", 3); private final String versionString; private final int fieldCount; @@ -71,6 +80,10 @@ public static TracingHeaderVersion getCurrentVersion() { return V2; } + public static TracingHeaderVersion getMetricsCurrentVersion() { + return AV0; + } + public int getFieldCount() { return fieldCount; } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadFooterMetrics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadFooterMetrics.java index ff5009785f4f9..ac229afd6558e 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadFooterMetrics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadFooterMetrics.java @@ -21,10 +21,7 @@ import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL_INFO; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_READ_BUFFER_SIZE; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_WRITE_BUFFER_SIZE; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRIC_ACCOUNT_KEY; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRIC_ACCOUNT_NAME; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRIC_FORMAT; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRIC_URI; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRICS_FORMAT; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_KB; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB; @@ -57,32 +54,14 @@ import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; -import static org.assertj.core.api.Assumptions.assumeThat; - public class ITestAbfsReadFooterMetrics extends AbstractAbfsScaleTest { - public ITestAbfsReadFooterMetrics() throws Exception { - checkPrerequisites(); - } - - private void checkPrerequisites(){ - checkIfConfigIsSet(FS_AZURE_METRIC_ACCOUNT_NAME); - checkIfConfigIsSet(FS_AZURE_METRIC_ACCOUNT_KEY); - checkIfConfigIsSet(FS_AZURE_METRIC_URI); - } - - private void checkIfConfigIsSet(String configKey) { - AbfsConfiguration conf = getConfiguration(); - String value = conf.get(configKey); - assumeThat(value) - .as(configKey + " config is mandatory for the test to run") - .isNotNull() - .matches(v -> v.trim().length() > 1, "trimmed length > 1"); - } - private static final String TEST_PATH = "/testfile"; private static final String SLEEP_PERIOD = "90000"; + public ITestAbfsReadFooterMetrics() throws Exception { + } + /** * Integration test for reading footer metrics with both Parquet and non-Parquet reads. */ @@ -99,7 +78,7 @@ public void testReadFooterMetricsWithParquetAndNonParquet() throws Exception { */ private Configuration getConfiguration(int bufferSize) { final Configuration configuration = getRawConfiguration(); - configuration.set(FS_AZURE_METRIC_FORMAT, String.valueOf(MetricFormat.INTERNAL_FOOTER_METRIC_FORMAT)); + configuration.set(FS_AZURE_METRICS_FORMAT, String.valueOf(MetricFormat.INTERNAL_FOOTER_METRIC_FORMAT)); configuration.setInt(AZURE_READ_BUFFER_SIZE, bufferSize); configuration.setInt(AZURE_WRITE_BUFFER_SIZE, bufferSize); return configuration; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java index 1eb21ce9da9d9..daa1bda78c2e7 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.azurebfs.services; +import java.io.FileNotFoundException; import java.io.IOException; import java.net.ProtocolException; import java.net.URI; @@ -38,6 +39,8 @@ import org.mockito.Mockito; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; @@ -49,19 +52,24 @@ import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType; import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; +import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; import org.apache.hadoop.fs.azurebfs.constants.HttpOperationType; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsApacheHttpExpect100Exception; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TokenAccessProviderException; import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters; +import org.apache.hadoop.fs.azurebfs.enums.AbfsBackoffMetricsEnum; import org.apache.hadoop.fs.azurebfs.extensions.SASTokenProvider; import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; +import org.apache.hadoop.fs.azurebfs.utils.MetricFormat; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat; +import org.apache.hadoop.fs.azurebfs.utils.UriUtils; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; import org.apache.hadoop.test.ReflectionUtils; import org.apache.http.HttpClientConnection; import org.apache.http.HttpResponse; +import org.apache.http.client.utils.URIBuilder; import static java.net.HttpURLConnection.HTTP_NOT_FOUND; import static org.apache.hadoop.fs.azurebfs.ITestAzureBlobFileSystemListStatus.TEST_CONTINUATION_TOKEN; @@ -72,9 +80,19 @@ import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EXPECT_100_JDK_ERROR; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.FORWARD_SLASH; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_HEAD; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PATCH; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PUT; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HUNDRED_CONTINUE; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_IS_HNS_ENABLED; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ALWAYS_USE_HTTPS; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRICS_COLLECTION_ENABLED; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRICS_EMIT_THRESHOLD_INTERVAL_SECS; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRICS_ACCOUNT_KEY; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRICS_ACCOUNT_NAME; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRICS_FORMAT; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRICS_EMIT_THRESHOLD; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.JAVA_VENDOR; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.JAVA_VERSION; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.OS_ARCH; @@ -82,13 +100,10 @@ import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.OS_VERSION; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.SEMICOLON; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.SINGLE_WHITE_SPACE; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_IS_HNS_ENABLED; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_APACHE_HTTP_CLIENT_CACHE_WARMUP_COUNT; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLUSTER_NAME; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLUSTER_TYPE; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRIC_ACCOUNT_NAME; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_VALUE_UNKNOWN; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.EXPECT; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE; @@ -99,8 +114,11 @@ import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME; import static org.apache.hadoop.fs.azurebfs.services.AuthType.SharedKey; +import static org.apache.hadoop.fs.azurebfs.services.RetryPolicyConstants.EXPONENTIAL_RETRY_POLICY_ABBREVIATION; +import static org.apache.hadoop.fs.azurebfs.utils.MetricFormat.INTERNAL_BACKOFF_METRIC_FORMAT; import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.assertj.core.api.Assumptions.assumeThat; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; @@ -890,6 +908,343 @@ public void testIsNonEmptyDirectory() throws IOException { false, 1, true); } + /** + * Test to verify that in case metric account is not set, + * metric collection is enabled with default metric format + * and account url. + * + * @throws Exception in case of any failure + */ + @Test + public void testMetricAccountFallback() throws Exception { + Configuration configuration = getRawConfiguration(); + configuration.setBoolean( + AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, false); + configuration.setBoolean(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, false); + configuration.unset(FS_AZURE_METRICS_ACCOUNT_NAME); + configuration.unset(FS_AZURE_METRICS_ACCOUNT_KEY); + configuration.unset(FS_AZURE_METRICS_FORMAT); + configuration.setBoolean(FS_AZURE_ALWAYS_USE_HTTPS, true); + final AzureBlobFileSystem fs = getFileSystem(configuration); + Assertions.assertThat( + fs.getAbfsStore().getAbfsConfiguration().getMetricFormat()) + .describedAs( + "In case metric format is not set, metric format should " + + "be defaulted to internal metric format") + .isEqualTo(MetricFormat.INTERNAL_METRIC_FORMAT); + + Assertions.assertThat( + fs.getAbfsStore().getClient().getAbfsMetricsManager().isMetricCollectionEnabled()) + .describedAs( + "Metric collection should be enabled even if metric account is not set") + .isTrue(); + + Assertions.assertThat( + fs.getAbfsStore().getClient().getAbfsCounters().toString()) + .describedAs( + "AbfsCounters should not contain backoff related metrics " + + "as no metric is collected for backoff") + .doesNotContain("#BO:"); + + Assertions.assertThat( + fs.getAbfsStore().getClient().getAbfsCounters().toString()) + .describedAs( + "AbfsCounters should not contain read footer related metrics " + + "as no metric is collected for read footer") + .doesNotContain("#FO:"); + + final URIBuilder uriBuilder = new URIBuilder(); + uriBuilder.setScheme(FileSystemUriSchemes.HTTPS_SCHEME); + uriBuilder.setHost(fs.getUri().getHost()); + uriBuilder.setPath(FORWARD_SLASH); + Assertions.assertThat(fs.getAbfsStore().getClient().getAbfsMetricsManager().getMetricsUrl()) + .describedAs( + "In case metric account is not set, account url should be used") + .isEqualTo( + UriUtils.changeUrlFromBlobToDfs(uriBuilder.build().toURL())); + } + + @Test + public void testInvalidMetricAccount() throws Exception { + Configuration configuration = getRawConfiguration(); + configuration.setBoolean( + AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, false); + configuration.setBoolean(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, false); + configuration.set(FS_AZURE_METRICS_ACCOUNT_NAME, "invalidAccountName!"); + configuration.set(FS_AZURE_METRICS_ACCOUNT_KEY, "invalidAccountKey!"); + configuration.unset(FS_AZURE_METRICS_FORMAT); // Use default metric format + // Filesystem initialization should not fail if metric account is invalid + try (AzureBlobFileSystem fs = getFileSystem(configuration)) { + // Incase metric account is invalid, metric collection should be disabled + Assertions.assertThat( + fs.getAbfsStore() + .getClient() + .getAbfsMetricsManager() + .isMetricCollectionEnabled()) + .describedAs( + "In case metric account is invalid, metric collection should be disabled") + .isFalse(); + fs.create(new Path("/testPath")); + FileStatus fileStatus = fs.getFileStatus(new Path("/testPath")); + Assertions.assertThat(fileStatus) + .describedAs("FileStatus should be returned for the created path") + .isNotNull(); + // Get metrics and it should be null + Assertions.assertThat( + fs.getAbfsStore().getClient().getAbfsCounters().toString()) + .describedAs( + "In case metric account is invalid, no metrics should be collected") + .doesNotContain("#BO:") + .doesNotContain("#FO:"); + } + } + + /** + * Test to verify that in case metric format is set to empty, + * metric collection is disabled. + * + * @throws Exception in case of any failure + */ + @Test + public void testMetricCollectionWithDifferentMetricFormat() throws Exception { + Configuration configuration = getRawConfiguration(); + // Setting this configuration just to ensure there is only one call during filesystem initialization + configuration.setBoolean( + AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true); + configuration.setBoolean(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, false); + configuration.unset(FS_AZURE_METRICS_FORMAT); + configuration.setEnum(FS_AZURE_METRICS_FORMAT, + INTERNAL_BACKOFF_METRIC_FORMAT); + final AzureBlobFileSystem fs = getFileSystem(configuration); + int totalCalls = 1; // Filesystem initialization call + Assertions.assertThat( + fs.getAbfsStore().getClient().getAbfsMetricsManager().isMetricCollectionEnabled()) + .describedAs("Metric collection should be enabled by default") + .isTrue(); + + Assertions.assertThat( + fs.getAbfsStore().getAbfsConfiguration().getMetricFormat()) + .describedAs("Metric format should be as set in configuration") + .isEqualTo(INTERNAL_BACKOFF_METRIC_FORMAT); + + Assertions.assertThat( + fs.getAbfsStore().getClient().getAbfsCounters().toString()) + .describedAs( + "AbfsCounters should only contains backoff related metrics when " + + "metric format is internal backoff metric format") + .contains("#BO:"); + + Assertions.assertThat( + fs.getAbfsStore().getClient().getAbfsCounters().toString()) + .describedAs( + "AbfsCounters should not contains read footer related metrics when " + + "metric format is internal backoff metric format") + .doesNotContain("#FO:"); + + Assertions.assertThat(fs.getAbfsStore() + .getClient() + .getAbfsCounters() + .getAbfsBackoffMetrics() + .getMetricValue( + AbfsBackoffMetricsEnum.TOTAL_NUMBER_OF_REQUESTS)) + .describedAs( + "Total number of requests should be 1 for filesystem initialization") + .isEqualTo(totalCalls); + + + if (fs.getAbfsStore().getClient() instanceof AbfsDfsClient) { + intercept(FileNotFoundException.class, + "The specified path does not exist.", + () -> fs.listStatus(path("/testPath"))); + totalCalls += 1; // listStatus call + } else { + intercept(FileNotFoundException.class, + "The specified blob does not exist.", + () -> fs.listStatus(path("/testPath"))); + totalCalls += 2; // listStatus call makes 2 calls to the service + } + + Assertions.assertThat(fs.getAbfsStore() + .getClient() + .getAbfsCounters() + .getAbfsBackoffMetrics() + .getMetricValue( + AbfsBackoffMetricsEnum.TOTAL_NUMBER_OF_REQUESTS)) + .describedAs( + "Total number of requests should be 2 after listStatus") + .isEqualTo(totalCalls); + } + + /** + * Test to verify that clientRequestId contains backoff metrics + * when metric format is set to internal backoff metric format. + * + * @throws Exception in case of any failure + */ + @Test + public void testGetMetricsCallMethod() throws Exception { + // File system init will make few calls to the service. + // Backoff metrics will be collected for those calls. + AzureBlobFileSystem fs = getFileSystem(); + TracingContext tracingContext = new TracingContext( + fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(), + "test-filesystem-id", FSOperationType.TEST_OP, true, + TracingHeaderFormat.AGGREGATED_METRICS_FORMAT, null, + fs.getAbfsStore().getClient().getAbfsCounters().toString()); + + AbfsHttpOperation abfsHttpOperation = getAbfsClient( + fs.getAbfsStore()).getAbfsRestOperation( + AbfsRestOperationType.GetFileSystemProperties, + HTTP_METHOD_HEAD, + fs.getAbfsStore().getClient().getAbfsMetricsManager().getMetricsUrl(), + getTestRequestHeaders(fs.getAbfsStore().getClient())) + .createHttpOperation(); + tracingContext.constructHeader(abfsHttpOperation, null, + EXPONENTIAL_RETRY_POLICY_ABBREVIATION); + assertThat(abfsHttpOperation.getClientRequestId()) + .describedAs("ClientRequestId should be containing Backoff metrics") + .contains("#BO:"); + } + + /** + * Verifies that metrics are emitted when the threshold is low. + */ + @Test + public void testMetricsEmitBasedOnCount() throws Exception { + final long threshold = 10L; + runMetricsEmitTest(threshold, true); + } + + /** + * Verifies that metrics are not emitted when the threshold is high. + */ + @Test + public void testMetricsEmitWithHighThreshold() throws Exception { + final long threshold = 100L; + runMetricsEmitTest(threshold, false); + } + + /** + * Runs a metrics emit test for a given threshold and expected behavior. + * Uses the same write/flush pattern and asserts based on emit expectation. + */ + private void runMetricsEmitTest(long threshold, boolean expectEmit) + throws Exception { + final int totalWaitTime = 30; + AzureBlobFileSystem fs = getFileSystem(); + Configuration configuration = fs.getAbfsStore() + .getAbfsConfiguration() + .getRawConfiguration(); + configuration.setLong(FS_AZURE_METRICS_EMIT_THRESHOLD, threshold); + configuration.setLong(FS_AZURE_METRICS_EMIT_THRESHOLD_INTERVAL_SECS, totalWaitTime); + fs = (AzureBlobFileSystem) FileSystem.newInstance(configuration); + + // Initial total metrics + long totalMetrics = fs.getAbfsStore().getClient().getAbfsCounters() + .getAbfsBackoffMetrics() + .getMetricValue(AbfsBackoffMetricsEnum.TOTAL_NUMBER_OF_REQUESTS); + + // Create file + Path testPath = path(TEST_PATH); + FSDataOutputStream stream = fs.create(testPath); + if (fs.getAbfsStore() + .getClientHandler() + .getIngressClient() instanceof AbfsDfsClient) { + // create file + set properties requests + totalMetrics += 1; + } else if (fs.getAbfsStore() + .getClient() instanceof AbfsDfsClient + && fs.getAbfsStore() + .getClientHandler() + .getIngressClient() instanceof AbfsBlobClient + && getIsNamespaceEnabled(fs)) { + totalMetrics += 2; + } else { + // create file + set properties + get properties requests + totalMetrics += 4; + } + + Assertions.assertThat(currentTotal(fs)) + .describedAs("Total number of requests should increase after create") + .isEqualTo(totalMetrics); + + // Append data + final int writeSize = 1024 * 1024; + final int numWrites = 10; + final byte dataByte = 5; + byte[] data = new byte[writeSize]; + Arrays.fill(data, dataByte); + + for (int i = 0; i < numWrites; i++) { + stream.write(data); // +1 request + stream.hflush(); + if (fs.getAbfsStore() + .isAppendBlobKey(fs.makeQualified(testPath).toString())) { + totalMetrics += 1; // +1 request + } else { + totalMetrics += 2; // +2 requests + } + } + + if (fs.getAbfsStore() + .getClientHandler() + .getIngressClient() instanceof AbfsDfsClient) { + totalMetrics += 1; // One extra call for flush in case of DFS client + } + + // Close stream + stream.close(); + + // Before waiting for emit scheduler to run, total metrics should match + Assertions.assertThat(currentTotal(fs)) + .describedAs( + "Total requests should match counted requests when threshold is high") + .isEqualTo(totalMetrics); + // Wait for emit scheduler to run + Thread.sleep(totalWaitTime * 1000); // 30 seconds + + if (expectEmit) { + Assertions.assertThat(currentTotal(fs)) + .describedAs( + "Once the emit scheduler has run, total requests should be reset to 0") + .isEqualTo(0); + } else { + Assertions.assertThat(currentTotal(fs)) + .describedAs( + "In case threshold is high, total requests should remain the same after emit scheduler run") + .isEqualTo(totalMetrics); + } + } + + @Test + public void testAggregateMetricsConfigs() throws Exception { + Configuration configuration = getRawConfiguration(); + // Disabling the aggregate metrics collection + configuration.setBoolean(FS_AZURE_METRICS_COLLECTION_ENABLED, false); + AzureBlobFileSystem fs = this.getFileSystem(configuration); + Assertions.assertThat(fs.getAbfsStore().getClient().getAbfsMetricsManager().getMetricsEmitScheduler()) + .describedAs("Since metrics collection is not enabled, " + + "scheduler should not be initialised") + .isNull(); + + // Disabling the aggregate metrics collection + configuration.setBoolean(FS_AZURE_METRICS_COLLECTION_ENABLED, true); + fs = this.getFileSystem(configuration); + Assertions.assertThat(fs.getAbfsStore().getClient().getAbfsMetricsManager().getMetricsEmitScheduler()) + .describedAs("Since metrics collection is not enabled, " + + "scheduler should initialised") + .isNotNull(); + } + + /** + * Returns the current total number of requests from AbfsBackoffMetrics. + */ + private long currentTotal(AzureBlobFileSystem fs) { + return fs.getAbfsStore().getClient().getAbfsCounters() + .getAbfsBackoffMetrics() + .getMetricValue(AbfsBackoffMetricsEnum.TOTAL_NUMBER_OF_REQUESTS); + } + private void testIsNonEmptyDirectoryInternal(String firstCT, boolean isfirstEmpty, String secondCT, boolean isSecondEmpty, int expectedInvocations, boolean isNonEmpty) throws IOException { @@ -1008,7 +1363,7 @@ public void testStaleConnectionBehavior() throws Exception { Configuration conf = fs.getConf(); // This is to avoid actual metric calls during the test - conf.unset(FS_AZURE_METRIC_ACCOUNT_NAME); + conf.unset(FS_AZURE_METRICS_ACCOUNT_NAME); // Initialize the file system AzureBlobFileSystemStore store = this.getFileSystem(conf).getAbfsStore(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestApacheClientConnectionPool.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestApacheClientConnectionPool.java index db5f596bad828..422d4cc945bb9 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestApacheClientConnectionPool.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestApacheClientConnectionPool.java @@ -51,7 +51,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.JDK_FALLBACK; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.KEEP_ALIVE_CACHE_CLOSED; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRIC_FORMAT; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRICS_FORMAT; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_NETWORKING_LIBRARY; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.HTTPS_SCHEME; import static org.apache.hadoop.fs.azurebfs.constants.HttpOperationType.APACHE_HTTP_CLIENT; @@ -74,7 +74,7 @@ public ITestApacheClientConnectionPool() throws Exception { public void testKacIsClosed() throws Throwable { Configuration configuration = new Configuration(getRawConfiguration()); configuration.set(FS_AZURE_NETWORKING_LIBRARY, APACHE_HTTP_CLIENT.name()); - configuration.unset(FS_AZURE_METRIC_FORMAT); + configuration.unset(FS_AZURE_METRICS_FORMAT); try (AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance( configuration)) { KeepAliveCache kac = fs.getAbfsStore() diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsBackoffMetrics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsBackoffMetrics.java index b8ce3ba411c0c..956548a8df670 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsBackoffMetrics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsBackoffMetrics.java @@ -40,7 +40,7 @@ public class TestAbfsBackoffMetrics { */ @BeforeEach public void setUp() { - metrics = new AbfsBackoffMetrics(); + metrics = new AbfsBackoffMetrics(true); } /** diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java index 039e01ed09581..8b0d7d728f562 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java @@ -21,6 +21,7 @@ import java.net.URI; import java.net.URL; import java.util.Map; +import java.util.UUID; import org.junit.jupiter.api.Test; @@ -29,13 +30,10 @@ import org.apache.hadoop.fs.azurebfs.AbfsCountersImpl; import org.apache.hadoop.fs.azurebfs.MockIntercept; import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; -import org.apache.hadoop.fs.azurebfs.utils.Base64; -import org.apache.hadoop.fs.azurebfs.utils.MetricFormat; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRIC_ACCOUNT_KEY; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRIC_ACCOUNT_NAME; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRIC_FORMAT; -import static org.apache.hadoop.fs.azurebfs.services.AbfsClient.ABFS_CLIENT_TIMER_THREAD_NAME; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRICS_FORMAT; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRICS_SHOULD_EMIT_ON_IDLE_TIME; +import static org.apache.hadoop.fs.azurebfs.services.AbfsMetricsManager.ABFS_CLIENT_TIMER_THREAD_NAME; import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doAnswer; @@ -59,13 +57,15 @@ public class TestAbfsClient { public void testTimerInitializationWithoutMetricCollection() throws Exception { final Configuration configuration = new Configuration(); AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, ACCOUNT_NAME); - abfsConfiguration.unset(FS_AZURE_METRIC_FORMAT); + abfsConfiguration.unset(FS_AZURE_METRICS_FORMAT); + configuration.setBoolean(FS_AZURE_METRICS_SHOULD_EMIT_ON_IDLE_TIME, false); AbfsCounters abfsCounters = spy(new AbfsCountersImpl(new URI("abcd"))); - AbfsClientContext abfsClientContext = new AbfsClientContextBuilder().withAbfsCounters(abfsCounters).build(); + AbfsClientContext abfsClientContext = new AbfsClientContextBuilder().withAbfsCounters(abfsCounters) + .withFileSystemId(UUID.randomUUID().toString()).build(); // Get an instance of AbfsClient. - AbfsClient client = new AbfsDfsClient(new URL("https://azure.com"), + AbfsClient client = new AbfsDfsClient(new URL("https://" + ACCOUNT_NAME + "/"), null, abfsConfiguration, (AccessTokenProvider) null, @@ -73,7 +73,7 @@ public void testTimerInitializationWithoutMetricCollection() throws Exception { null, abfsClientContext); - assertThat(client.getTimer()) + assertThat(client.getAbfsMetricsManager().getTimer()) .describedAs("Timer should not be initialized") .isNull(); @@ -93,16 +93,15 @@ public void testTimerInitializationWithoutMetricCollection() throws Exception { @Test public void testTimerInitializationWithMetricCollection() throws Exception { final Configuration configuration = new Configuration(); - configuration.set(FS_AZURE_METRIC_FORMAT, String.valueOf(MetricFormat.INTERNAL_BACKOFF_METRIC_FORMAT)); - configuration.set(FS_AZURE_METRIC_ACCOUNT_NAME, ACCOUNT_NAME); - configuration.set(FS_AZURE_METRIC_ACCOUNT_KEY, Base64.encode(ACCOUNT_KEY.getBytes())); + configuration.setBoolean(FS_AZURE_METRICS_SHOULD_EMIT_ON_IDLE_TIME, true); AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, ACCOUNT_NAME); AbfsCounters abfsCounters = spy(new AbfsCountersImpl(new URI("abcd"))); - AbfsClientContext abfsClientContext = new AbfsClientContextBuilder().withAbfsCounters(abfsCounters).build(); + AbfsClientContext abfsClientContext = new AbfsClientContextBuilder().withAbfsCounters(abfsCounters) + .withFileSystemId(UUID.randomUUID().toString()).build(); // Get an instance of AbfsClient. - AbfsClient client = new AbfsDfsClient(new URL("https://azure.com"), + AbfsClient client = new AbfsDfsClient(new URL("https://" + ACCOUNT_NAME + "/"), null, abfsConfiguration, (AccessTokenProvider) null, @@ -110,7 +109,7 @@ public void testTimerInitializationWithMetricCollection() throws Exception { null, abfsClientContext); - assertThat(client.getTimer()) + assertThat(client.getAbfsMetricsManager().getTimer()) .describedAs("Timer should be initialized") .isNotNull(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRenameRetryRecovery.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRenameRetryRecovery.java index c9f3b34f90911..98d876f5f332b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRenameRetryRecovery.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRenameRetryRecovery.java @@ -231,7 +231,7 @@ private void executeThenFail(final AbfsClient client, final byte[] buffer = answer.getArgument(0); final int offset = answer.getArgument(1); final int length = answer.getArgument(2); - normalRestOp.signRequest(normalOp, length); + normalRestOp.signRequest(normalOp, length, false); normalOp.sendPayload(buffer, offset, length); normalOp.processResponse(buffer, offset, length); LOG.info("Actual outcome is {} \"{}\" \"{}\"; injecting failure", diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java index 62f6e253fb518..352108de11f51 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java @@ -26,10 +26,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_DELETE; import static org.apache.hadoop.fs.azurebfs.enums.AbfsBackoffMetricsEnum.NUMBER_OF_REQUESTS_FAILED; import static org.apache.hadoop.fs.azurebfs.services.AbfsRestOperationType.DeletePath; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRIC_ACCOUNT_KEY; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRIC_ACCOUNT_NAME; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRIC_FORMAT; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRIC_URI; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRICS_FORMAT; import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; import java.util.ArrayList; @@ -42,12 +39,6 @@ public class TestAbfsRestOperation extends public TestAbfsRestOperation() throws Exception { } - private void checkPrerequisites() { - assumeValidTestConfigPresent(getRawConfiguration(), FS_AZURE_METRIC_ACCOUNT_NAME); - assumeValidTestConfigPresent(getRawConfiguration(), FS_AZURE_METRIC_ACCOUNT_KEY); - assumeValidTestConfigPresent(getRawConfiguration(), FS_AZURE_METRIC_URI); - } - /** * Test for backoff retry metrics. * @@ -58,10 +49,9 @@ private void checkPrerequisites() { */ @Test public void testBackoffRetryMetrics() throws Exception { - checkPrerequisites(); // Create an AzureBlobFileSystem instance. final Configuration configuration = getRawConfiguration(); - configuration.set(FS_AZURE_METRIC_FORMAT, String.valueOf(MetricFormat.INTERNAL_BACKOFF_METRIC_FORMAT)); + configuration.set(FS_AZURE_METRICS_FORMAT, String.valueOf(MetricFormat.INTERNAL_BACKOFF_METRIC_FORMAT)); final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(configuration); AbfsConfiguration abfsConfiguration = fs.getAbfsStore().getAbfsConfiguration(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAggregateMetricsManager.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAggregateMetricsManager.java new file mode 100644 index 0000000000000..6058f95687137 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAggregateMetricsManager.java @@ -0,0 +1,718 @@ +/** + * 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.hadoop.fs.azurebfs.services; + +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardCopyOption; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; + +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; + +import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore; + +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COLON; + +public class TestAggregateMetricsManager extends AbstractAbfsIntegrationTest { + + // Number of nanoseconds in one millisecond. + private static final long NANOS_PER_MILLISECOND = 1_000_000L; + + // The manager under test + private final AggregateMetricsManager manager; + + // Rate limit permits per second for testing + private final int permitsPerSecond = 3; + + /** + * Constructor for TestAggregateMetricsManager. + * + * @throws Exception if an error occurs during setup + */ + public TestAggregateMetricsManager() throws Exception { + super(); + manager = AggregateMetricsManager.getInstance(10, permitsPerSecond); + } + + /** + * Creates a fully instrumented TracingContext useful for metric dispatch tests. + */ + private String getMetricsData() { + return UUID.randomUUID() + COLON + UUID.randomUUID() + COLON + "#BO:"; + } + + /** + * Wraps an AbfsClient instance in a spy and counts metric call invocations. + */ + private AbfsClient spyClient(AzureBlobFileSystem azureBlobFileSystem, + AtomicInteger counter) + throws IOException { + AzureBlobFileSystemStore store = Mockito.spy( + azureBlobFileSystem.getAbfsStore()); + Mockito.doReturn(store).when(azureBlobFileSystem).getAbfsStore(); + AbfsClient client = Mockito.spy(store.getClient()); + Mockito.doReturn(client).when(store).getClient(); + + Mockito.doAnswer(inv -> { + counter.incrementAndGet(); + return null; + }).when(client).getMetricCall(Mockito.any()); + + return client; + } + + /** + * Verifies that multiple recordMetric calls result in exactly one aggregated + * dispatch within a 1-second dispatch window. + */ + @Test + public void testRecordMetric() throws Exception { + AtomicInteger calls = new AtomicInteger(0); + AbfsClient client = spyClient(Mockito.spy(this.getFileSystem()), calls); + manager.registerClient("acc1", client); + for (int i = 0; i < 5; i++) { + manager.recordMetric("acc1", getMetricsData()); + } + manager.deregisterClient("acc1", client); + + Assertions.assertThat(calls.get()) + .describedAs("Expected exactly 1 aggregated metrics send") + .isEqualTo(1); + } + + /** + * Verifies that aggregated metrics for the same account are sent + * once per dispatch window, meaning 2 windows → 2 sends. + */ + @Test + public void testRecordMetricTwoWindows() throws Exception { + AtomicInteger calls = new AtomicInteger(0); + AbfsClient client = spyClient(Mockito.spy(this.getFileSystem()), calls); + manager.registerClient("acc1", client); + for (int i = 0; i < 5; i++) { + manager.recordMetric("acc1", getMetricsData()); + } + + manager.deregisterClient("acc1", client); + Assertions.assertThat(calls.get()) + .describedAs("Expected 1 aggregated sends") + .isEqualTo(1); + + manager.registerClient("acc1", client); + // Second window + for (int i = 0; i < 5; i++) { + manager.recordMetric("acc1", getMetricsData()); + } + manager.deregisterClient("acc1", client); + + Assertions.assertThat(calls.get()) + .describedAs("Expected 2 aggregated sends") + .isEqualTo(2); + } + + /** + * Ensures that recordMetric handles invalid input without exceptions. + */ + @Test + public void testRecordMetricWithNulls() throws Exception { + manager.recordMetric(null, null); + manager.recordMetric("", null); + manager.recordMetric("acc", null); + manager.recordMetric("acc", null); + } + + /** + * Ensures that metrics for separate accounts still respect global + * rate limiting but send independently within the same window. + */ + @Test + public void testMultipleAccounts() throws Exception { + + AtomicInteger calls1 = new AtomicInteger(); + AbfsClient client1 = spyClient(Mockito.spy(this.getFileSystem()), calls1); + + AtomicInteger calls2 = new AtomicInteger(); + AbfsClient client2 = spyClient(Mockito.spy( + (AzureBlobFileSystem) AzureBlobFileSystem.newInstance( + getRawConfiguration())), calls2); + + manager.registerClient("acc1", client1); + manager.registerClient("acc2", client2); + manager.recordMetric("acc1", getMetricsData()); + manager.recordMetric("acc2", getMetricsData()); + manager.deregisterClient("acc1", client1); + manager.deregisterClient("acc2", client2); + + Assertions.assertThat(calls1.get()) + .describedAs("Account 1 dispatched once") + .isEqualTo(1); + + Assertions.assertThat(calls2.get()) + .describedAs("Account 2 dispatched once") + .isEqualTo(1); + } + + /** + * Tests concurrent registration, metric recording, and deregistration + * of multiple clients for the same account. + */ + @Test + public void testMultipleClientsRegistryInParallel() throws Exception { + AtomicInteger calls1 = new AtomicInteger(0); + AbfsClient client1 = spyClient(Mockito.spy(this.getFileSystem()), calls1); + + AtomicInteger calls2 = new AtomicInteger(0); + AbfsClient client2 = spyClient(Mockito.spy(this.getFileSystem()), calls2); + + CountDownLatch latch = new CountDownLatch(5); + + new Thread(() -> { + try { + manager.registerClient("acc1", client1); + } finally { + latch.countDown(); + } + }).start(); + + new Thread(() -> { + try { + manager.registerClient("acc1", client2); + } finally { + latch.countDown(); + } + }).start(); + + new Thread(() -> { + try { + for (int i = 0; i < 10; i++) { + manager.recordMetric("acc1", getMetricsData()); + } + } finally { + latch.countDown(); + } + }).start(); + + new Thread(() -> { + try { + manager.deregisterClient("acc1", client1); + } finally { + latch.countDown(); + } + }).start(); + + new Thread(() -> { + try { + manager.deregisterClient("acc1", client2); + } finally { + latch.countDown(); + } + }).start(); + + //wait for all threads to finish + latch.await(); + + Assertions.assertThat((calls1.get() == 1) ^ (calls2.get() == 1)) + .describedAs("Exactly one client should send metrics") + .isTrue(); + } + + /** + * Tests deregistering a nonexistent client. + */ + @Test + public void testDeregisterNonexistentClient() throws IOException { + AbfsClient client = Mockito.spy( + this.getFileSystem().getAbfsStore().getClient()); + // Should not throw + boolean isRemoved = manager.deregisterClient("nonexistentAccount", client); + Assertions.assertThat(isRemoved) + .describedAs("Deregistering nonexistent client should return false") + .isFalse(); + } + + /** + * Tests that when the aggregated metric data exceeds the buffer size, + * multiple dispatches occur as expected. + */ + @Test + public void testMultipleMetricCallsInCaseDataIsMoreThanBufferSize() + throws Exception { + final int metricsDataSize1 + = 927; // size of aggregated data for first 3 calls + final int metricsDataSize2 = 115; // size of aggregated data for last call + final int numberOfMetrics = 25; // total metrics to send + AtomicInteger calls = new AtomicInteger(0); + AzureBlobFileSystem azureBlobFileSystem = Mockito.spy(this.getFileSystem()); + AzureBlobFileSystemStore store = Mockito.spy( + azureBlobFileSystem.getAbfsStore()); + Mockito.doReturn(store).when(azureBlobFileSystem).getAbfsStore(); + AbfsClient client = Mockito.spy(store.getClient()); + Mockito.doReturn(client).when(store).getClient(); + + Mockito.doAnswer(inv -> { + String data = (String) inv.getArguments()[0]; + if (calls.get() < 3) { // first three calls, data size will be 927 chars + Assertions.assertThat(data.length()) + .describedAs("Aggregated metric data size should be 927 chars") + .isEqualTo(metricsDataSize1); + } else { // last call, data size will be 115 chars + Assertions.assertThat(data.length()) + .describedAs("Aggregated metric data size should be 115 chars") + .isEqualTo(metricsDataSize2); + } + calls.incrementAndGet(); + return null; + }).when(client).getMetricCall(Mockito.any()); + manager.registerClient("acc1", client); + for (int i = 0; i < numberOfMetrics; i++) { + manager.recordMetric("acc1", getMetricsData() + + "$OT=163$RT=6.024%$TRNR=2543$TR=2706"); // each data is 113 chars + } + manager.deregisterClient("acc1", client); + + // 113 + 2 ([,]) = 115 chars per metric, 115 * 25 = 2875 chars total + 24 (:) = 2899 chars + // 1st -> 115 * 8 = 920 chars + 7 (:) = 927 chars + // 2nd -> 115 * 8 = 920 chars + 7 (:) = 927 chars + // 3rd -> 115 * 9 = 920 chars + 7 (:) = 927 chars + // 4th -> remaining + Assertions.assertThat(calls.get()) + .describedAs("Expected exactly 3 aggregated metrics send") + .isEqualTo(4); + } + + /** + * Verifies that when multiple clients send metrics concurrently, + * the global rate limiter enforces spacing between dispatches. + */ + @Test + public void testRateLimitMetricCalls() + throws IOException, InterruptedException { + final long minIntervalMs = 1_000 / permitsPerSecond; // 333ms + final double toleranceMs = 50; // allow 50ms jitter + final int numClients = 10; + + // Store timestamps for each client + final List times = new ArrayList<>(); + AbfsClient[] abfsClients = new AbfsClient[numClients]; + + for (int i = 0; i < numClients; i++) { + AtomicLong time = new AtomicLong(); + times.add(time); + + AbfsClient client = createSpiedClient(time); + abfsClients[i] = client; + manager.registerClient("acc" + i, client); + } + + // Record metrics for all clients + for (int i = 0; i < numClients; i++) { + manager.recordMetric("acc" + i, getMetricsData()); + } + + // Deregister all clients concurrently + CountDownLatch latch = new CountDownLatch(numClients); + for (int i = 0; i < numClients; i++) { + final int idx = i; + new Thread(() -> { + try { + manager.deregisterClient("acc" + idx, + abfsClients[idx]); // pass time for demonstration if needed + } finally { + latch.countDown(); + } + }).start(); + } + latch.await(); + + // Check that interval between any two timestamps is ≥ minIntervalMs - tolerance + for (int i = 0; i < times.size(); i++) { + for (int j = i + 1; j < times.size(); j++) { + double diffMs = Math.abs(times.get(i).get() - times.get(j).get()) + / (double) NANOS_PER_MILLISECOND; + Assertions.assertThat(diffMs) + .describedAs( + "Expected at least %d ms (tolerance %.3f) between metric sends", + minIntervalMs, toleranceMs) + .isGreaterThanOrEqualTo(minIntervalMs - toleranceMs); + } + } + } + + + /** + * Tests that the shutdown hook flushes metrics on JVM exit. + */ + @Test + public void testAggregatedMetricsManagerWithJVMExit0() + throws IOException, InterruptedException { + String program = + "import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;\n" + + "import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;\n" + + "import org.apache.hadoop.fs.azurebfs.services.*;\n" + + "import org.apache.hadoop.fs.azurebfs.utils.*;\n" + + "import org.apache.hadoop.conf.Configuration;\n" + + "import org.apache.hadoop.fs.FileSystem;\n" + + "import java.util.*;\n" + + "import java.util.concurrent.atomic.AtomicInteger;\n" + + "import java.io.IOException;\n" + + "import java.net.URI;\n" + + "import org.mockito.Mockito;\n" + + "\n" + + "public class ShutdownTestProg {\n" + + " public static void main(String[] args) throws Exception {\n" + + " AtomicInteger calls1 = new AtomicInteger();\n" + + " AggregateMetricsManager mgr = AggregateMetricsManager.getInstance(10, 3);\n" + + "\n" + + " URI uri = new URI(\"abfss://test@manishtestfnsnew.dfs.core.windows.net\");\n" + + " Configuration config = new Configuration();\n" + + "\n" + + " AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(uri, config);\n" + + " AbfsClient client = spyClient(Mockito.spy(fs), calls1);\n" + + "\n" + + " mgr.registerClient(\"acc1\", client);\n" + + " mgr.recordMetric(\"acc1\", \"m1\");\n" + + " mgr.recordMetric(\"acc1\", \"m2\");\n" + + "\n" + + " System.out.println(\"BEFORE_EXIT\");\n" + + " System.exit(0);\n" + + " }\n" + + "\n" + + " private static AbfsClient spyClient(AzureBlobFileSystem azureBlobFileSystem,\n" + + " AtomicInteger counter) throws IOException {\n" + + "\n" + + " AzureBlobFileSystemStore store = Mockito.spy(azureBlobFileSystem.getAbfsStore());\n" + + " Mockito.doReturn(store).when(azureBlobFileSystem).getAbfsStore();\n" + + "\n" + + " AbfsClient client = Mockito.spy(store.getClient());\n" + + " Mockito.doReturn(client).when(store).getClient();\n" + + "\n" + + " Mockito.doAnswer(inv -> {\n" + + " counter.incrementAndGet();\n" + + " System.out.println(\"FLUSH:\" + inv.getArguments()[0]);\n" + + " return null;\n" + + " }).when(client).getMetricCall(Mockito.any());\n" + + "\n" + + " return client;\n" + + " }\n" + + "}\n"; + + runProgramAndCaptureOutput(program, true, 0); + } + + /** + * Tests that the shutdown hook flushes metrics on JVM exit after multiple + * clients and deregistrations. + */ + @Test + public void testAggregatedMetricsManagerWithJVMExit1() + throws IOException, InterruptedException { + String program = + "import org.apache.hadoop.fs.azurebfs.services.*;\n" + + "import org.apache.hadoop.fs.azurebfs.utils.*;\n" + + "import org.apache.hadoop.conf.Configuration;\n" + + "import org.apache.hadoop.fs.FileSystem;\n" + + "import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;\n" + + "import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;\n" + + "import java.util.*;\n" + + "import java.util.concurrent.atomic.AtomicInteger;\n" + + "import java.io.IOException;\n" + + "import java.net.URI;\n" + + "import org.mockito.Mockito;\n" + + "\n" + + "public class ShutdownTestProg {\n" + + " public static void main(String[] args) throws Exception {\n" + + " AggregateMetricsManager mgr = AggregateMetricsManager.getInstance(10, 3);\n" + + "\n" + + " AtomicInteger calls1 = new AtomicInteger();\n" + + " AtomicInteger calls2 = new AtomicInteger();\n" + + " AtomicInteger calls3 = new AtomicInteger();\n" + + "\n" + + " URI uri = new URI(\"abfss://test@manishtestfnsnew.dfs.core.windows.net\");\n" + + " Configuration config = new Configuration();\n" + + "\n" + + " AzureBlobFileSystem fs1 = (AzureBlobFileSystem) FileSystem.newInstance(uri, config);\n" + + " AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(uri, config);\n" + + " AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.newInstance(uri, config);\n" + + "\n" + + " AbfsClient client1 = spyClient(Mockito.spy(fs1), calls1);\n" + + " AbfsClient client2 = spyClient(Mockito.spy(fs2), calls2);\n" + + " AbfsClient client3 = spyClient(Mockito.spy(fs3), calls3);\n" + + "\n" + + " mgr.registerClient(\"acc1\", client1);\n" + + " mgr.registerClient(\"acc1\", client2);\n" + + " mgr.registerClient(\"acc1\", client3);\n" + + "\n" + + " mgr.recordMetric(\"acc1\", \"m1\");\n" + + " mgr.recordMetric(\"acc1\", \"m2\");\n" + + "\n" + + " mgr.recordMetric(\"acc1\", \"m3\");\n" + + " mgr.recordMetric(\"acc1\", \"m4\");\n" + + "\n" + + " mgr.recordMetric(\"acc1\", \"m5\");\n" + + " mgr.recordMetric(\"acc1\", \"m6\");\n" + + "\n" + + " System.out.println(\"BEFORE_EXIT\");\n" + + " mgr.deregisterClient(\"acc1\", client3);\n" + + " mgr.deregisterClient(\"acc1\", client2);\n" + + " mgr.deregisterClient(\"acc1\", client1);\n" + + " System.out.println(\"BEFORE_EXIT1\");\n" + + " System.exit(1);\n" + + " }\n" + + "\n" + + " private static AbfsClient spyClient(AzureBlobFileSystem azureBlobFileSystem,\n" + + " AtomicInteger counter) throws IOException {\n" + + "\n" + + " AzureBlobFileSystemStore store = Mockito.spy(azureBlobFileSystem.getAbfsStore());\n" + + " Mockito.doReturn(store).when(azureBlobFileSystem).getAbfsStore();\n" + + "\n" + + " AbfsClient client = Mockito.spy(store.getClient());\n" + + " Mockito.doReturn(client).when(store).getClient();\n" + + "\n" + + " Mockito.doAnswer(inv -> {\n" + + " counter.incrementAndGet();\n" + + " System.out.println(\"FLUSH:\" + inv.getArguments()[0]);\n" + + " return null;\n" + + " }).when(client).getMetricCall(Mockito.any());\n" + + "\n" + + " return client;\n" + + " }\n" + + "}\n"; + + runProgramAndCaptureOutput(program, true, 1); + } + + /** + * Tests that the shutdown hook does not flush metrics on JVM crash. + */ + @Test + void testAggregatedMetricsManagerWithJVMCrash() throws Exception { + final int crashExitCode = 134; + String program = + "import org.apache.hadoop.fs.azurebfs.services.*;\n" + + "import org.apache.hadoop.fs.azurebfs.utils.*;\n" + + "import org.apache.hadoop.conf.Configuration;\n" + + "import org.apache.hadoop.fs.FileSystem;\n" + + "import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;\n" + + "import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;\n" + + "import java.util.*;\n" + + "import java.util.concurrent.atomic.AtomicInteger;\n" + + "import java.io.IOException;\n" + + "import java.net.URI;\n" + + "import java.lang.reflect.*;\n" + + "import org.mockito.Mockito;\n" + + "\n" + + "public class ShutdownTestProg {\n" + + "\n" + + " public static void main(String[] args) throws Exception {\n" + + " AggregateMetricsManager mgr = AggregateMetricsManager.getInstance(10, 3);\n" + + "\n" + + " // Track how many times metrics flush\n" + + " AtomicInteger calls1 = new AtomicInteger();\n" + + " AtomicInteger calls2 = new AtomicInteger();\n" + + " AtomicInteger calls3 = new AtomicInteger();\n" + + "\n" + + " URI uri = new URI(\"abfss://test@manishtestfnsnew.dfs.core.windows.net\");\n" + + " Configuration config = new Configuration();\n" + + "\n" + + " // Initialize 3 separate file system instances\n" + + " AzureBlobFileSystem fs1 = (AzureBlobFileSystem) FileSystem.newInstance(uri, config);\n" + + " AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(uri, config);\n" + + " AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.newInstance(uri, config);\n" + + "\n" + + " // Create 3 spy clients\n" + + " AbfsClient client1 = spyClient(Mockito.spy(fs1), calls1);\n" + + " AbfsClient client2 = spyClient(Mockito.spy(fs2), calls2);\n" + + " AbfsClient client3 = spyClient(Mockito.spy(fs3), calls3);\n" + + "\n" + + " mgr.registerClient(\"acc1\", client1);\n" + + " mgr.registerClient(\"acc2\", client2);\n" + + " mgr.registerClient(\"acc3\", client3);\n" + + "\n" + + " // Produce metrics on all clients\n" + + " mgr.recordMetric(\"acc1\", \"m1\");\n" + + " mgr.recordMetric(\"acc1\", \"m2\");\n" + + " mgr.recordMetric(\"acc2\", \"m3\");\n" + + " mgr.recordMetric(\"acc2\", \"m4\");\n" + + " mgr.recordMetric(\"acc3\", \"m5\");\n" + + " mgr.recordMetric(\"acc3\", \"m6\");\n" + + "\n" + + " System.out.println(\"BEFORE_EXIT\");\n" + + " crashJvm();\n" + + " }\n" + + "\n" + + " private static void crashJvm() throws Exception {\n" + + " Field f = sun.misc.Unsafe.class.getDeclaredField(\"theUnsafe\");\n" + + " f.setAccessible(true);\n" + + " sun.misc.Unsafe unsafe = (sun.misc.Unsafe) f.get(null);\n" + + " unsafe.putAddress(0, 0); // SIGSEGV → Immediate JVM crash\n" + // 128 + 6 = 134 (exitcode for SIGABRT) + + " }\n" + + "\n" + + " private static AbfsClient spyClient(AzureBlobFileSystem azureBlobFileSystem, AtomicInteger counter) throws IOException {\n" + + " AzureBlobFileSystemStore store = Mockito.spy(azureBlobFileSystem.getAbfsStore());\n" + + " Mockito.doReturn(store).when(azureBlobFileSystem).getAbfsStore();\n" + + " AbfsClient client = Mockito.spy(store.getClient());\n" + + " Mockito.doReturn(client).when(store).getClient();\n" + + "\n" + + " Mockito.doAnswer(inv -> {\n" + + " counter.incrementAndGet();\n" + + " System.out.println(\"FLUSH:\" + inv.getArguments()[0]);\n" + + " return null;\n" + + " }).when(client).getMetricCall(Mockito.any());\n" + + "\n" + + " return client;\n" + + " }\n" + + "}\n"; + + runProgramAndCaptureOutput(program, false, crashExitCode); + } + + /** + * Compiles and runs a Java program in a separate JVM, capturing its output. + * + * @param program The Java program source code as a string. + * @param expectMetricsFlush Whether to expect metrics flush output. + * @throws IOException If an I/O error occurs. + * @throws InterruptedException If the thread is interrupted while waiting. + */ + private void runProgramAndCaptureOutput(String program, + boolean expectMetricsFlush, int expectedExitCode) + throws IOException, InterruptedException { + final long waitTimeInSeconds = 30; + Path tempFile = Files.createTempFile("ShutdownTestProg", ".java"); + try { + Files.write(tempFile, program.getBytes(StandardCharsets.UTF_8)); + + Path javaFile = tempFile.getParent().resolve("ShutdownTestProg.java"); + Files.move(tempFile, javaFile, StandardCopyOption.REPLACE_EXISTING); + + Process javac = new ProcessBuilder( + "javac", + "-classpath", System.getProperty("java.class.path"), + javaFile.toAbsolutePath().toString()) + .redirectErrorStream(true) + .start(); + + String compileOutput = readProcessOutput(javac); + javac.waitFor(); + if (!javac.waitFor(waitTimeInSeconds, TimeUnit.SECONDS)) { + javac.destroyForcibly(); + throw new AssertionError("java process timed out"); + } + + Assertions.assertThat(javac.exitValue()) + .withFailMessage("Compilation failed:\n" + compileOutput) + .isEqualTo(0); + + String classpath = javaFile.getParent().toAbsolutePath() + + File.pathSeparator + + System.getProperty("java.class.path"); + + Process javaProc = new ProcessBuilder("java", + "-XX:ErrorFile=/tmp/no_hs_err_%p.log", + "-classpath", classpath, + "ShutdownTestProg") + .redirectErrorStream(true) + .start(); + + String output = readProcessOutput(javaProc); + int exitCode; + if (!javaProc.waitFor(waitTimeInSeconds, TimeUnit.SECONDS)) { + javaProc.destroyForcibly(); + throw new AssertionError("java process timed out"); + } + exitCode = javaProc.exitValue(); + + Assertions.assertThat(output).contains("BEFORE_EXIT"); + Assertions.assertThat(exitCode).isEqualTo(expectedExitCode); + + if (expectMetricsFlush) { + Assertions.assertThat(output).contains("FLUSH:"); + } else { + Assertions.assertThat(output).doesNotContain("FLUSH:"); + } + } finally { + Files.deleteIfExists(tempFile); + } + } + + /** + * Reads all output from a process's input stream. + * + * @param proc The process to read from. + * @return The output as a string. + * @throws IOException If an I/O error occurs. + * @throws InterruptedException If the thread is interrupted while waiting. + */ + private static String readProcessOutput(Process proc) + throws IOException, InterruptedException { + final int maxBufferSize = 4096; + ByteArrayOutputStream out = new ByteArrayOutputStream(); + Thread t = new Thread(() -> { + try (InputStream in = proc.getInputStream()) { + byte[] buf = new byte[maxBufferSize]; + int n; + while ((n = in.read(buf)) != -1) { + out.write(buf, 0, n); + } + } catch (IOException e) { + e.printStackTrace(); + } + }); + t.start(); + int exitCode = proc.waitFor(); // wait for process to exit + t.join(); // wait for reading thread to finish + return out.toString(String.valueOf(StandardCharsets.UTF_8)); + } + + /** + * Helper method to create a spied client and record timestamp on getMetricCall + */ + private AbfsClient createSpiedClient(AtomicLong time) throws IOException { + AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem()); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + + AbfsClient client = Mockito.spy(store.getClient()); + Mockito.doReturn(client).when(store).getClient(); + + Mockito.doAnswer(inv -> { + time.set(System.nanoTime()); + return null; + }).when(client).getMetricCall(Mockito.any()); + + return client; + } + +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestApacheHttpClientFallback.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestApacheHttpClientFallback.java index 55c5f76c71b52..42b27a0111a3b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestApacheHttpClientFallback.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestApacheHttpClientFallback.java @@ -177,7 +177,8 @@ private AbfsRestOperation getMockRestOperation(int[] retryIteration) Mockito.doNothing() .when(op) - .signRequest(Mockito.any(AbfsHttpOperation.class), Mockito.anyInt()); + .signRequest(Mockito.any(AbfsHttpOperation.class), Mockito.anyInt(), + Mockito.anyBoolean()); Mockito.doAnswer(answer -> { AbfsHttpOperation operation = Mockito.spy( diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TestSimpleRateLimiter.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TestSimpleRateLimiter.java new file mode 100644 index 0000000000000..aa2481fc19d58 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TestSimpleRateLimiter.java @@ -0,0 +1,172 @@ +/** + * 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.hadoop.fs.azurebfs.utils; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.LockSupport; + +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; + +public class TestSimpleRateLimiter { + + /** Number of nanoseconds in one millisecond. */ + private static final long NANOS_PER_MILLISECOND = 1_000_000L; + + /** + * Verifies that the rate limiter does not introduce unnecessary blocking + * when calls are naturally spaced apart longer than the required interval. + * + * The test creates a limiter allowing 2 permits per second (500ms + * interval). After calling {@code acquire()}, it waits 600ms—longer than + * required—so the next {@code acquire()} should return immediately. + */ + @Test + void testNoWaitWhenSpacedOut() throws InvalidConfigurationValueException { + final int millisToSleep = 600; // 600 ms + // 2 permits per second → 500 ms interval + SimpleRateLimiter limiter = new SimpleRateLimiter(2); + + limiter.acquire(5, TimeUnit.SECONDS); + // Sleep longer than required interval + LockSupport.parkNanos(millisToSleep * NANOS_PER_MILLISECOND); + + long before = System.nanoTime(); + limiter.acquire(5, TimeUnit.SECONDS); // Should not block + long after = System.nanoTime(); + + long elapsed = after - before; + + // Should be less than 5ms + Assertions.assertThat(elapsed < 5 * NANOS_PER_MILLISECOND) + .describedAs("acquire() should not block when enough time has passed") + .isTrue(); + } + + /** + * Verifies that the rate limiter enforces the correct delay when + * {@code acquire()} is called faster than the configured rate. + * + * At 5 permits per second (200ms interval), two immediate consecutive + * calls should cause the second call to block for roughly 200ms. + */ + @Test + void testRateLimitingDelay() throws InvalidConfigurationValueException { + final int permitsPerSecond = 5; + final long minTimeAllowed = 180; + final long maxTimeAllowed = 260; + // 5 permits per second → 200ms interval + SimpleRateLimiter limiter = new SimpleRateLimiter(permitsPerSecond); + + limiter.acquire(5, TimeUnit.SECONDS); // First call never waits + + long before = System.nanoTime(); + limiter.acquire(5, TimeUnit.SECONDS); // Second call immediately → should wait ~200ms + long after = System.nanoTime(); + + long elapsedMs = (after - before) / NANOS_PER_MILLISECOND; + + // Expect ~200ms, so allow tolerance + Assertions.assertThat( + elapsedMs >= minTimeAllowed && elapsedMs <= maxTimeAllowed) + .describedAs("Expected about 200ms wait, but was " + elapsedMs + " ms") + .isTrue(); + } + + /** + * Tests that multiple rapid calls produce cumulative waiting consistent + * with the configured permit interval. + * + * At 10 permits per second (100ms interval), five immediate calls should + * take around 400ms total (the first call is free; the remaining four + * require spacing). + */ + @Test + void testMultipleBurstCalls() + throws InvalidConfigurationValueException, InterruptedException { + final int permitsPerSecond = 10; + final long minTimeAllowed = 350; + final long maxTimeAllowed = 550; + // 10 permits per second → 100ms interval + SimpleRateLimiter limiter = new SimpleRateLimiter(permitsPerSecond); + + long totalStart = System.nanoTime(); + + CountDownLatch latch = new CountDownLatch(5); + for (int i = 0; i < 5; i++) { + new Thread(() -> { + try { + limiter.acquire(5, TimeUnit.SECONDS); + } finally { + latch.countDown(); + } + }).start(); + } + //wait for all threads to finish + latch.await(); + + long totalMs = (System.nanoTime() - totalStart) / NANOS_PER_MILLISECOND; + + // 5 calls → should take around 400ms (first is free, next 4 need +100ms each) + Assertions.assertThat( + totalMs >= minTimeAllowed && totalMs <= maxTimeAllowed) + .describedAs("Expected around 400ms total but got " + totalMs + "ms") + .isTrue(); + } + + /** + * Verifies that when 10 rapid acquire() calls are made with a rate limit + * of 3 permits per second (≈333ms interval), the total execution time is + * spread across ~3 seconds, since each call must be spaced by the interval. + * + * Expected timing: + * interval = 333ms + * first call: no wait + * remaining 9 calls must wait: 9 × 333ms ≈ 2997ms + * + * Total expected time: around 3.0 seconds. + */ + @Test + void testMultipleBurstCallsWhenPermitIsLess() + throws InvalidConfigurationValueException { + final int permitsPerSecond = 3; + final long minTimeAllowed = 2700; + final long maxTimeAllowed = 3500; + SimpleRateLimiter limiter = new SimpleRateLimiter(permitsPerSecond); + + long start = System.nanoTime(); + + for (int i = 0; i < 10; i++) { + limiter.acquire(5, TimeUnit.SECONDS); + } + + long end = System.nanoTime(); + long elapsedMs = (end - start) / NANOS_PER_MILLISECOND; + + // Expected ~3000ms, allow tolerance due to scheduler delays. + Assertions.assertThat( + elapsedMs >= minTimeAllowed && elapsedMs <= maxTimeAllowed) + .describedAs("Expected ~3000ms, but got " + elapsedMs + "ms") + .isTrue(); + } +} +