HADOOP-14475 Metrics of S3A don't print out when enabled. Contributed by Younger and Sean Mackrory.
Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/6555af81 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6555af81 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6555af81 Branch: refs/heads/HDFS-7240 Commit: 6555af81a26b0b72ec3bee7034e01f5bd84b1564 Parents: a957f1c Author: Aaron Fabbri <[email protected]> Authored: Tue Dec 5 11:06:32 2017 -0800 Committer: Aaron Fabbri <[email protected]> Committed: Tue Dec 5 11:06:32 2017 -0800 ---------------------------------------------------------------------- .../org/apache/hadoop/fs/s3a/S3AFileSystem.java | 3 + .../hadoop/fs/s3a/S3AInstrumentation.java | 107 ++++++++++++++++--- .../src/site/markdown/tools/hadoop-aws/index.md | 38 ++++++- .../apache/hadoop/fs/s3a/ITestS3AMetrics.java | 51 +++++++++ 4 files changed, 186 insertions(+), 13 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hadoop/blob/6555af81/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java ---------------------------------------------------------------------- diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index fba8ea1..63a4349 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -78,6 +78,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.util.concurrent.ListeningExecutorService; +import org.apache.commons.io.IOUtils; import org.apache.commons.lang.StringUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -2423,6 +2424,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities { metadataStore.close(); metadataStore = null; } + IOUtils.closeQuietly(instrumentation); + instrumentation = null; } } http://git-wip-us.apache.org/repos/asf/hadoop/blob/6555af81/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java ---------------------------------------------------------------------- diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java index ef5a434..0fbcc00 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.s3a; +import com.google.common.annotations.VisibleForTesting; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -29,8 +30,10 @@ import org.apache.hadoop.metrics2.MetricStringBuilder; import org.apache.hadoop.metrics2.MetricsCollector; import org.apache.hadoop.metrics2.MetricsInfo; import org.apache.hadoop.metrics2.MetricsRecordBuilder; +import org.apache.hadoop.metrics2.MetricsSource; +import org.apache.hadoop.metrics2.MetricsSystem; import org.apache.hadoop.metrics2.MetricsTag; -import org.apache.hadoop.metrics2.annotation.Metrics; +import org.apache.hadoop.metrics2.impl.MetricsSystemImpl; import org.apache.hadoop.metrics2.lib.Interns; import org.apache.hadoop.metrics2.lib.MetricsRegistry; import org.apache.hadoop.metrics2.lib.MutableCounterLong; @@ -58,16 +61,49 @@ import static org.apache.hadoop.fs.s3a.Statistic.*; * the operations to increment/query metric values are designed to handle * lookup failures. */ -@Metrics(about = "Metrics for S3a", context = "S3AFileSystem") @InterfaceAudience.Private @InterfaceStability.Evolving -public class S3AInstrumentation { +public class S3AInstrumentation implements Closeable, MetricsSource { private static final Logger LOG = LoggerFactory.getLogger( S3AInstrumentation.class); - public static final String CONTEXT = "S3AFileSystem"; + private static final String METRICS_SOURCE_BASENAME = "S3AMetrics"; + + /** + * {@value #METRICS_SYSTEM_NAME} The name of the s3a-specific metrics + * system instance used for s3a metrics. + */ + public static final String METRICS_SYSTEM_NAME = "s3a-file-system"; + + /** + * {@value #CONTEXT} Currently all s3a metrics are placed in a single + * "context". Distinct contexts may be used in the future. + */ + public static final String CONTEXT = "s3aFileSystem"; + + /** + * {@value #METRIC_TAG_FILESYSTEM_ID} The name of a field added to metrics + * records that uniquely identifies a specific FileSystem instance. + */ + public static final String METRIC_TAG_FILESYSTEM_ID = "s3aFileSystemId"; + + /** + * {@value #METRIC_TAG_BUCKET} The name of a field added to metrics records + * that indicates the hostname portion of the FS URL. + */ + public static final String METRIC_TAG_BUCKET = "bucket"; + + // metricsSystemLock must be used to synchronize modifications to + // metricsSystem and the following counters. + private static Object metricsSystemLock = new Object(); + private static MetricsSystem metricsSystem = null; + private static int metricsSourceNameCounter = 0; + private static int metricsSourceActiveCounter = 0; + + private String metricsSourceName; + private final MetricsRegistry registry = - new MetricsRegistry("S3AFileSystem").setContext(CONTEXT); + new MetricsRegistry("s3aFileSystem").setContext(CONTEXT); private final MutableCounterLong streamOpenOperations; private final MutableCounterLong streamCloseOperations; private final MutableCounterLong streamClosed; @@ -146,7 +182,6 @@ public class S3AInstrumentation { STORE_IO_THROTTLED }; - private static final Statistic[] GAUGES_TO_CREATE = { OBJECT_PUT_REQUESTS_ACTIVE, OBJECT_PUT_BYTES_PENDING, @@ -157,12 +192,10 @@ public class S3AInstrumentation { public S3AInstrumentation(URI name) { UUID fileSystemInstanceId = UUID.randomUUID(); - registry.tag("FileSystemId", - "A unique identifier for the FS ", - fileSystemInstanceId.toString() + "-" + name.getHost()); - registry.tag("fsURI", - "URI of this filesystem", - name.toString()); + registry.tag(METRIC_TAG_FILESYSTEM_ID, + "A unique identifier for the instance", + fileSystemInstanceId.toString()); + registry.tag(METRIC_TAG_BUCKET, "Hostname from the FS URL", name.getHost()); streamOpenOperations = streamCounter(STREAM_OPENED); streamCloseOperations = streamCounter(STREAM_CLOSE_OPERATIONS); streamClosed = streamCounter(STREAM_CLOSED); @@ -204,6 +237,39 @@ public class S3AInstrumentation { "ops", "latency", interval); quantiles(S3GUARD_METADATASTORE_THROTTLE_RATE, "events", "frequency (Hz)", interval); + + registerAsMetricsSource(name); + } + + @VisibleForTesting + public MetricsSystem getMetricsSystem() { + synchronized (metricsSystemLock) { + if (metricsSystem == null) { + metricsSystem = new MetricsSystemImpl(); + metricsSystem.init(METRICS_SYSTEM_NAME); + } + } + return metricsSystem; + } + + /** + * Register this instance as a metrics source. + * @param name s3a:// URI for the associated FileSystem instance + */ + private void registerAsMetricsSource(URI name) { + int number; + synchronized(metricsSystemLock) { + getMetricsSystem(); + + metricsSourceActiveCounter++; + number = ++metricsSourceNameCounter; + } + String msName = METRICS_SOURCE_BASENAME + number; + if (number > 1) { + msName = msName + number; + } + metricsSourceName = msName + "-" + name.getHost(); + metricsSystem.register(metricsSourceName, "", this); } /** @@ -560,6 +626,23 @@ public class S3AInstrumentation { streamBytesDiscardedInAbort.incr(statistics.bytesDiscardedInAbort); } + @Override + public void getMetrics(MetricsCollector collector, boolean all) { + registry.snapshot(collector.addRecord(registry.info().name()), true); + } + + public void close() { + synchronized (metricsSystemLock) { + metricsSystem.unregisterSource(metricsSourceName); + int activeSources = --metricsSourceActiveCounter; + if (activeSources == 0) { + metricsSystem.publishMetricsNow(); + metricsSystem.shutdown(); + metricsSystem = null; + } + } + } + /** * Statistics updated by an input stream during its actual operation. * These counters not thread-safe and are for use in a single instance http://git-wip-us.apache.org/repos/asf/hadoop/blob/6555af81/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md ---------------------------------------------------------------------- diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md index aaf9f23..a8d2e48 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md @@ -1,4 +1,3 @@ - <!--- Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. @@ -1599,6 +1598,43 @@ basis. to set fadvise policies on input streams. Once implemented, this will become the supported mechanism used for configuring the input IO policy. +##<a name="metrics"></a>Metrics + +S3A metrics can be monitored through Hadoop's metrics2 framework. S3A creates +its own metrics system called s3a-file-system, and each instance of the client +will create its own metrics source, named with a JVM-unique numerical ID. + +As a simple example, the following can be added to `hadoop-metrics2.properties` +to write all S3A metrics to a log file every 10 seconds: + + s3a-file-system.sink.my-metrics-config.class=org.apache.hadoop.metrics2.sink.FileSink + s3a-file-system.sink.my-metrics-config.filename=/var/log/hadoop-yarn/s3a-metrics.out + *.period=10 + +Lines in that file will be structured like the following: + + 1511208770680 s3aFileSystem.s3aFileSystem: Context=s3aFileSystem, s3aFileSystemId=892b02bb-7b30-4ffe-80ca-3a9935e1d96e, bucket=bucket, + Hostname=hostname-1.hadoop.apache.com, files_created=1, files_copied=2, files_copied_bytes=10000, files_deleted=5, fake_directories_deleted=3, + directories_created=3, directories_deleted=0, ignored_errors=0, op_copy_from_local_file=0, op_exists=0, op_get_file_status=15, op_glob_status=0, + op_is_directory=0, op_is_file=0, op_list_files=0, op_list_located_status=0, op_list_status=3, op_mkdirs=1, op_rename=2, object_copy_requests=0, + object_delete_requests=6, object_list_requests=23, object_continue_list_requests=0, object_metadata_requests=46, object_multipart_aborted=0, + object_put_bytes=0, object_put_requests=4, object_put_requests_completed=4, stream_write_failures=0, stream_write_block_uploads=0, + stream_write_block_uploads_committed=0, stream_write_block_uploads_aborted=0, stream_write_total_time=0, stream_write_total_data=0, + s3guard_metadatastore_put_path_request=10, s3guard_metadatastore_initialization=0, object_put_requests_active=0, object_put_bytes_pending=0, + stream_write_block_uploads_active=0, stream_write_block_uploads_pending=0, stream_write_block_uploads_data_pending=0, + S3guard_metadatastore_put_path_latencyNumOps=0, S3guard_metadatastore_put_path_latency50thPercentileLatency=0, + S3guard_metadatastore_put_path_latency75thPercentileLatency=0, S3guard_metadatastore_put_path_latency90thPercentileLatency=0, + S3guard_metadatastore_put_path_latency95thPercentileLatency=0, S3guard_metadatastore_put_path_latency99thPercentileLatency=0 + +Depending on other configuration, metrics from other systems, contexts, etc. may +also get recorded, for example the following: + + 1511208770680 metricssystem.MetricsSystem: Context=metricssystem, Hostname=s3a-metrics-4.gce.cloudera.com, NumActiveSources=1, NumAllSources=1, + NumActiveSinks=1, NumAllSinks=0, Sink_fileNumOps=2, Sink_fileAvgTime=1.0, Sink_fileDropped=0, Sink_fileQsize=0, SnapshotNumOps=5, + SnapshotAvgTime=0.0, PublishNumOps=2, PublishAvgTime=0.0, DroppedPubAll=0 + +Note that low-level metrics from the AWS SDK itself are not currently included +in these metrics. ##<a name="further_reading"></a> Other Topics http://git-wip-us.apache.org/repos/asf/hadoop/blob/6555af81/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java ---------------------------------------------------------------------- diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java new file mode 100644 index 0000000..182990c --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java @@ -0,0 +1,51 @@ +/* + * 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.s3a; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.metrics2.lib.MutableCounterLong; +import org.junit.Test; + +import java.io.IOException; + +/** + * Test s3a performance metrics register and output. + */ +public class ITestS3AMetrics extends AbstractS3ATestBase { + + @Test + public void testMetricsRegister() + throws IOException, InterruptedException { + S3AFileSystem fs = getFileSystem(); + Path dest = new Path("newfile1"); + ContractTestUtils.touch(fs, dest); + + String targetMetricSource = "S3AMetrics1" + "-" + fs.getBucket(); + assertNotNull("No metrics under test fs for " + targetMetricSource, + fs.getInstrumentation().getMetricsSystem() + .getSource(targetMetricSource)); + + MutableCounterLong fileCreated = + (MutableCounterLong) fs.getInstrumentation().getRegistry() + .get(Statistic.FILES_CREATED.getSymbol()); + assertEquals("Metrics system should report single file created event", + 1, fileCreated.value()); + } +} --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
