result = new CompletableFuture<>();
try {
result.complete(callable.apply());
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/WrappedIOException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/WrappedIOException.java
index 1de1ecb785368..0ad4cda13da3b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/WrappedIOException.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/WrappedIOException.java
@@ -25,18 +25,21 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.util.functional.RuntimeIOException;
/**
* A wrapper for an IOException which
* {@link FutureIOSupport#raiseInnerCause(ExecutionException)} knows to
* always extract the exception.
- *
+ *
* The constructor signature guarantees the cause will be an IOException,
* and as it checks for a null-argument, non-null.
+ * @deprecated: use the public {@link RuntimeIOException}.
*/
+@Deprecated
@InterfaceAudience.Private
@InterfaceStability.Unstable
-public class WrappedIOException extends RuntimeException {
+public class WrappedIOException extends RuntimeIOException {
private static final long serialVersionUID = 2510210974235779294L;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/DurationTracker.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/DurationTracker.java
new file mode 100644
index 0000000000000..9d1f6f42df199
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/DurationTracker.java
@@ -0,0 +1,33 @@
+/*
+ * 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.statistics;
+
+/**
+ * Interface to be implemented by objects which can track duration.
+ * It extends AutoCloseable to fit into a try-with-resources statement,
+ * but then strips out the {@code throws Exception} aspect of the signature
+ * so it doesn't force code to add extra handling for any failures.
+ */
+public interface DurationTracker extends AutoCloseable {
+
+ /**
+ * Finish tracking: update the statistics with the timings.
+ */
+ void close();
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/DurationTrackerFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/DurationTrackerFactory.java
new file mode 100644
index 0000000000000..c9c3fcc9c2c0e
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/DurationTrackerFactory.java
@@ -0,0 +1,58 @@
+/*
+ * 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.statistics;
+
+/**
+ * Interface for a source of duration tracking.
+ *
+ * This is intended for uses where it can be passed into classes
+ * which update operation durations, without tying those
+ * classes to internal implementation details.
+ */
+public interface DurationTrackerFactory {
+
+ /**
+ * Initiate a duration tracking operation by creating/returning
+ * an object whose {@code close()} call will
+ * update the statistics.
+ *
+ * The statistics counter with the key name will be incremented
+ * by the given count.
+ *
+ * The expected use is within a try-with-resources clause.
+ * @param key statistic key prefix
+ * @param count #of times to increment the matching counter in this
+ * operation.
+ * @return an object to close after an operation completes.
+ */
+ DurationTracker trackDuration(String key, int count);
+
+ /**
+ * Initiate a duration tracking operation by creating/returning
+ * an object whose {@code close()} call will
+ * update the statistics.
+ *
+ * The expected use is within a try-with-resources clause.
+ * @param key statistic key
+ * @return an object to close after an operation completes.
+ */
+ default DurationTracker trackDuration(String key) {
+ return trackDuration(key, 1);
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatistics.java
new file mode 100644
index 0000000000000..75d9965128101
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatistics.java
@@ -0,0 +1,78 @@
+/*
+ * 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.statistics;
+
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * IO Statistics.
+ *
+ * These are low-cost per-instance statistics provided by any Hadoop
+ * I/O class instance.
+ *
+ * Consult the filesystem specification document for the requirements
+ * of an implementation of this interface.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public interface IOStatistics {
+
+ /**
+ * Map of counters.
+ * @return the current map of counters.
+ */
+ Map counters();
+
+ /**
+ * Map of gauges.
+ * @return the current map of gauges.
+ */
+ Map gauges();
+
+ /**
+ * Map of minimums.
+ * @return the current map of minimums.
+ */
+ Map minimums();
+
+ /**
+ * Map of maximums.
+ * @return the current map of maximums.
+ */
+ Map maximums();
+
+ /**
+ * Map of meanStatistics.
+ * @return the current map of MeanStatistic statistics.
+ */
+ Map meanStatistics();
+
+ /**
+ * Value when a minimum value has never been set.
+ */
+ long MIN_UNSET_VALUE = -1;
+
+ /**
+ * Value when a max value has never been set.
+ */
+ long MAX_UNSET_VALUE = -1;
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsAggregator.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsAggregator.java
new file mode 100644
index 0000000000000..1c5451c6f0e83
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsAggregator.java
@@ -0,0 +1,48 @@
+/*
+ * 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.statistics;
+
+import javax.annotation.Nullable;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Interface exported by classes which support
+ * aggregation of {@link IOStatistics}.
+ * Implementations MAY aggregate all statistics
+ * exported by the IOStatistics reference passed in to
+ * {@link #aggregate(IOStatistics)}, or they
+ * may selectively aggregate specific values/classes
+ * of statistics.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface IOStatisticsAggregator {
+
+ /**
+ * Aggregate the supplied statistics into the current
+ * set.
+ *
+ * @param statistics statistics; may be null
+ * @return true if the statistics reference was not null and
+ * so aggregated.
+ */
+ boolean aggregate(@Nullable IOStatistics statistics);
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsLogging.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsLogging.java
new file mode 100644
index 0000000000000..42dd28082874d
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsLogging.java
@@ -0,0 +1,207 @@
+/*
+ * 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.statistics;
+
+import javax.annotation.Nullable;
+import java.util.Map;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding;
+
+import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics;
+
+/**
+ * Utility operations convert IO Statistics sources/instances
+ * to strings, especially for robustly logging.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public final class IOStatisticsLogging {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(IOStatisticsLogging.class);
+
+ private IOStatisticsLogging() {
+ }
+
+ /**
+ * Extract the statistics from a source object -or ""
+ * if it is not an instance of {@link IOStatistics},
+ * {@link IOStatisticsSource} or the retrieved
+ * statistics are null.
+ *
+ * Exceptions are caught and downgraded to debug logging.
+ * @param source source of statistics.
+ * @return a string for logging.
+ */
+ public static String ioStatisticsSourceToString(@Nullable Object source) {
+ try {
+ return ioStatisticsToString(retrieveIOStatistics(source));
+ } catch (RuntimeException e) {
+ LOG.debug("Ignoring", e);
+ return "";
+ }
+ }
+
+ /**
+ * Convert IOStatistics to a string form.
+ * @param statistics A statistics instance.
+ * @return string value or the empty string if null
+ */
+ public static String ioStatisticsToString(
+ @Nullable final IOStatistics statistics) {
+ if (statistics != null) {
+ StringBuilder sb = new StringBuilder();
+ mapToString(sb, "counters", statistics.counters());
+ mapToString(sb, "gauges", statistics.gauges());
+ mapToString(sb, "minimums", statistics.minimums());
+ mapToString(sb, "maximums", statistics.maximums());
+ mapToString(sb, "means", statistics.meanStatistics());
+
+ return sb.toString();
+ } else {
+ return "";
+ }
+ }
+
+ private static void mapToString(StringBuilder sb,
+ final String type,
+ final Map map) {
+ int count = 0;
+ sb.append(type);
+ sb.append("=(");
+ for (Map.Entry entry : map.entrySet()) {
+ if (count > 0) {
+ sb.append(' ');
+ }
+ count++;
+ sb.append(IOStatisticsBinding.entryToString(
+ entry.getKey(), entry.getValue()));
+ }
+ sb.append("); ");
+ }
+
+ /**
+ * On demand stringifier of an IOStatisticsSource instance.
+ *
+ * Whenever this object's toString() method is called, it evaluates the
+ * statistics.
+ *
+ * This is designed to affordable to use in log statements.
+ * @param source source of statistics -may be null.
+ * @return an object whose toString() operation returns the current values.
+ */
+ public static Object demandStringifyIOStatisticsSource(
+ @Nullable IOStatisticsSource source) {
+ return new SourceToString(source);
+ }
+
+ /**
+ * On demand stringifier of an IOStatistics instance.
+ *
+ * Whenever this object's toString() method is called, it evaluates the
+ * statistics.
+ *
+ * This is for use in log statements where for the cost of creation
+ * of this entry is low; it is affordable to use in log statements.
+ * @param statistics statistics to stringify -may be null.
+ * @return an object whose toString() operation returns the current values.
+ */
+ public static Object demandStringifyIOStatistics(
+ @Nullable IOStatistics statistics) {
+ return new StatisticsToString(statistics);
+ }
+
+ /**
+ * Extract any statistics from the source and log at debug, if
+ * the log is set to log at debug.
+ * No-op if logging is not at debug or the source is null/of
+ * the wrong type/doesn't provide statistics.
+ * @param log log to log to
+ * @param message message for log -this must contain "{}" for the
+ * statistics report
+ * @param source source object
+ */
+ public static void logIOStatisticsAtDebug(
+ Logger log,
+ String message,
+ Object source) {
+ if (log.isDebugEnabled()) {
+ // robust extract and convert to string
+ String stats = ioStatisticsSourceToString(source);
+ if (!stats.isEmpty()) {
+ log.debug(message, stats);
+ }
+ }
+ }
+
+ /**
+ * On demand stringifier.
+ *
+ * Whenever this object's toString() method is called, it
+ * retrieves the latest statistics instance and re-evaluates it.
+ */
+ private static final class SourceToString {
+
+ private final IOStatisticsSource source;
+
+ private SourceToString(@Nullable IOStatisticsSource source) {
+ this.source = source;
+ }
+
+ @Override
+ public String toString() {
+ return source != null
+ ? ioStatisticsSourceToString(source)
+ : IOStatisticsBinding.NULL_SOURCE;
+ }
+ }
+
+ /**
+ * Stringifier of statistics: low cost to instantiate and every
+ * toString/logging will re-evaluate the statistics.
+ */
+ private static final class StatisticsToString {
+
+ private final IOStatistics statistics;
+
+ /**
+ * Constructor.
+ * @param statistics statistics
+ */
+ private StatisticsToString(@Nullable IOStatistics statistics) {
+ this.statistics = statistics;
+ }
+
+ /**
+ * Evaluate and stringify the statistics.
+ * @return a string value.
+ */
+ @Override
+ public String toString() {
+ return statistics != null
+ ? ioStatisticsToString(statistics)
+ : IOStatisticsBinding.NULL_SOURCE;
+ }
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSnapshot.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSnapshot.java
new file mode 100644
index 0000000000000..b4d91327a11ad
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSnapshot.java
@@ -0,0 +1,215 @@
+/*
+ * 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.statistics;
+
+import javax.annotation.Nullable;
+import java.io.Serializable;
+import java.util.Map;
+import java.util.TreeMap;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding;
+import org.apache.hadoop.util.JsonSerialization;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToString;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.aggregateMaps;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.snapshotMap;
+
+/**
+ * Snapshot of statistics from a different source.
+ *
+ * It is serializable so that frameworks which can use java serialization
+ * to propagate data (Spark, Flink...) can send the statistics
+ * back. For this reason, TreeMaps are explicitly used as field types,
+ * even though IDEs can recommend use of Map instead.
+ *
+ * It is annotated for correct serializations with jackson2.
+ */
+@SuppressWarnings("CollectionDeclaredAsConcreteClass")
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class IOStatisticsSnapshot
+ implements IOStatistics, Serializable, IOStatisticsAggregator {
+
+ private static final long serialVersionUID = -1762522703841538084L;
+
+ /**
+ * Counters.
+ */
+ @JsonProperty
+ private TreeMap counters;
+
+ /**
+ * Gauges.
+ */
+ @JsonProperty
+ private TreeMap gauges;
+
+ /**
+ * Minimum values.
+ */
+ @JsonProperty
+ private TreeMap minimums;
+
+ /**
+ * Maximum values.
+ */
+ @JsonProperty
+ private TreeMap maximums;
+
+ /**
+ * mean statistics. The JSON key is all lower case..
+ */
+ @JsonProperty("meanstatistics")
+ private TreeMap meanStatistics;
+
+ /**
+ * Construct.
+ */
+ public IOStatisticsSnapshot() {
+ createMaps();
+ }
+
+ /**
+ * Construct, taking a snapshot of the source statistics data
+ * if the source is non-null.
+ * If the source is null, the empty maps are created
+ * @param source statistics source. Nullable.
+ */
+ public IOStatisticsSnapshot(IOStatistics source) {
+ if (source != null) {
+ snapshot(source);
+ } else {
+ createMaps();
+ }
+ }
+
+ /**
+ * Create the maps.
+ */
+ private void createMaps() {
+ counters = new TreeMap<>();
+ gauges = new TreeMap<>();
+ minimums = new TreeMap<>();
+ maximums = new TreeMap<>();
+ meanStatistics = new TreeMap<>();
+ }
+
+ /**
+ * Clear all the maps.
+ */
+ public synchronized void clear() {
+ counters.clear();
+ gauges.clear();
+ minimums.clear();
+ maximums.clear();
+ meanStatistics.clear();
+ }
+
+ /**
+ * Take a snapshot.
+ *
+ * This completely overwrites the map data with the statistics
+ * from the source.
+ * @param source statistics source.
+ */
+ public synchronized void snapshot(IOStatistics source) {
+ checkNotNull(source);
+ counters = snapshotMap(source.counters());
+ gauges = snapshotMap(source.gauges());
+ minimums = snapshotMap(source.minimums());
+ maximums = snapshotMap(source.maximums());
+ meanStatistics = snapshotMap(source.meanStatistics(),
+ MeanStatistic::copy);
+ }
+
+ /**
+ * Aggregate the current statistics with the
+ * statistics reference passed in.
+ *
+ * The operation is synchronized.
+ * @param statistics statistics; may be null
+ * @return true if a merge took place.
+ */
+ @Override
+ public synchronized boolean aggregate(
+ @Nullable IOStatistics statistics) {
+ if (statistics == null) {
+ return false;
+ }
+ aggregateMaps(counters, statistics.counters(),
+ IOStatisticsBinding::aggregateCounters,
+ IOStatisticsBinding::passthroughFn);
+ aggregateMaps(gauges, statistics.gauges(),
+ IOStatisticsBinding::aggregateGauges,
+ IOStatisticsBinding::passthroughFn);
+ aggregateMaps(minimums, statistics.minimums(),
+ IOStatisticsBinding::aggregateMinimums,
+ IOStatisticsBinding::passthroughFn);
+ aggregateMaps(maximums, statistics.maximums(),
+ IOStatisticsBinding::aggregateMaximums,
+ IOStatisticsBinding::passthroughFn);
+ aggregateMaps(meanStatistics, statistics.meanStatistics(),
+ IOStatisticsBinding::aggregateMeanStatistics, MeanStatistic::copy);
+ return true;
+ }
+
+ @Override
+ public Map counters() {
+ return counters;
+ }
+
+ @Override
+ public Map gauges() {
+ return gauges;
+ }
+
+ @Override
+ public Map minimums() {
+ return minimums;
+ }
+
+ @Override
+ public Map maximums() {
+ return maximums;
+ }
+
+ @Override
+ public Map meanStatistics() {
+ return meanStatistics;
+ }
+
+ @Override
+ public String toString() {
+ return ioStatisticsToString(this);
+ }
+
+ /**
+ * Get a JSON serializer for this class.
+ * @return a serializer.
+ */
+ public static JsonSerialization serializer() {
+ return new JsonSerialization<>(IOStatisticsSnapshot.class, false, true);
+ }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSource.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSource.java
new file mode 100644
index 0000000000000..8b0287ad20952
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSource.java
@@ -0,0 +1,46 @@
+/*
+ * 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.statistics;
+
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A source of IO statistics.
+ *
+ * These statistics MUST be instance specific, not thread local.
+ */
+
+@InterfaceStability.Unstable
+public interface IOStatisticsSource {
+
+ /**
+ * Return a statistics instance.
+ *
+ * It is not a requirement that the same instance is returned every time.
+ * {@link IOStatisticsSource}.
+ *
+ * If the object implementing this is Closeable, this method
+ * may return null if invoked on a closed object, even if
+ * it returns a valid instance when called earlier.
+ * @return an IOStatistics instance or null
+ */
+ default IOStatistics getIOStatistics() {
+ return null;
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSupport.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSupport.java
new file mode 100644
index 0000000000000..3eaeccd547947
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSupport.java
@@ -0,0 +1,107 @@
+/*
+ * 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.statistics;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.statistics.impl.StubDurationTracker;
+import org.apache.hadoop.fs.statistics.impl.StubDurationTrackerFactory;
+
+/**
+ * Support for working with IOStatistics.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public final class IOStatisticsSupport {
+
+ private IOStatisticsSupport() {
+ }
+
+ /**
+ * Take a snapshot of the current statistics state.
+ *
+ * This is not an atomic option.
+ *
+ * The instance can be serialized, and its
+ * {@code toString()} method lists all the values.
+ * @param statistics statistics
+ * @return a snapshot of the current values.
+ */
+ public static IOStatisticsSnapshot
+ snapshotIOStatistics(IOStatistics statistics) {
+
+ return new IOStatisticsSnapshot(statistics);
+ }
+
+ /**
+ * Create a snapshot statistics instance ready to aggregate data.
+ *
+ * The instance can be serialized, and its
+ * {@code toString()} method lists all the values.
+ * @return an empty snapshot
+ */
+ public static IOStatisticsSnapshot
+ snapshotIOStatistics() {
+
+ return new IOStatisticsSnapshot();
+ }
+
+ /**
+ * Get the IOStatistics of the source, casting it
+ * if it is of the relevant type, otherwise,
+ * if it implements {@link IOStatisticsSource}
+ * extracting the value.
+ *
+ * Returns null if the source isn't of the write type
+ * or the return value of
+ * {@link IOStatisticsSource#getIOStatistics()} was null.
+ * @return an IOStatistics instance or null
+ */
+
+ public static IOStatistics retrieveIOStatistics(
+ final Object source) {
+ if (source instanceof IOStatistics) {
+ return (IOStatistics) source;
+ } else if (source instanceof IOStatisticsSource) {
+ return ((IOStatisticsSource) source).getIOStatistics();
+ } else {
+ // null source or interface not implemented
+ return null;
+ }
+ }
+
+ /**
+ * Return a stub duration tracker factory whose returned trackers
+ * are always no-ops.
+ *
+ * As singletons are returned, this is very low-cost to use.
+ * @return a duration tracker factory.
+ */
+ public static DurationTrackerFactory stubDurationTrackerFactory() {
+ return StubDurationTrackerFactory.STUB_DURATION_TRACKER_FACTORY;
+ }
+
+ /**
+ * Get a stub duration tracker.
+ * @return a stub tracker.
+ */
+ public static DurationTracker stubDurationTracker() {
+ return StubDurationTracker.STUB_DURATION_TRACKER;
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/MeanStatistic.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/MeanStatistic.java
new file mode 100644
index 0000000000000..f390170422d54
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/MeanStatistic.java
@@ -0,0 +1,278 @@
+/*
+ * 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.statistics;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A mean statistic represented as the sum and the sample count;
+ * the mean is calculated on demand.
+ *
+ * It can be used to accrue values so as to dynamically update
+ * the mean. If so, know that there is no synchronization
+ * on the methods.
+ *
+ * If a statistic has 0 samples then it is considered to be empty.
+ *
+ * All 'empty' statistics are equivalent, independent of the sum value.
+ *
+ * For non-empty statistics, sum and sample values must match
+ * for equality.
+ *
+ * It is serializable and annotated for correct serializations with jackson2.
+ *
+ * Thread safety. The operations to add/copy sample data, are thread safe.
+ *
+ * - {@link #add(MeanStatistic)}
+ * - {@link #addSample(long)}
+ * - {@link #clear()}
+ * - {@link #setSamplesAndSum(long, long)}
+ * - {@link #set(MeanStatistic)}
+ * - {@link #setSamples(long)} and {@link #setSum(long)}
+ *
+ * So is the {@link #mean()} method. This ensures that when
+ * used to aggregated statistics, the aggregate value and sample
+ * count are set and evaluated consistently.
+ *
+ * Other methods marked as synchronized because Findbugs overreacts
+ * to the idea that some operations to update sum and sample count
+ * are synchronized, but that things like equals are not.
+ *
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class MeanStatistic implements Serializable, Cloneable {
+
+ private static final long serialVersionUID = 567888327998615425L;
+
+ /**
+ * Number of samples used to calculate
+ * the mean.
+ */
+ private long samples;
+
+ /**
+ * sum of the values.
+ */
+ private long sum;
+
+ /**
+ * Constructor, with some resilience against invalid sample counts.
+ * If the sample count is 0 or less, the sum is set to 0 and
+ * the sample count to 0.
+ * @param samples sample count.
+ * @param sum sum value
+ */
+ public MeanStatistic(final long samples, final long sum) {
+ if (samples > 0) {
+ this.sum = sum;
+ this.samples = samples;
+ }
+ }
+
+ /**
+ * Create from another statistic.
+ * @param that source
+ */
+ public MeanStatistic(MeanStatistic that) {
+ this(that.samples, that.sum);
+ }
+
+ /**
+ * Create an empty statistic.
+ */
+ public MeanStatistic() {
+ }
+
+ /**
+ * Get the sum of samples.
+ * @return the sum
+ */
+ public long getSum() {
+ return sum;
+ }
+
+ /**
+ * Get the sample count.
+ * @return the sample count; 0 means empty
+ */
+ public long getSamples() {
+ return samples;
+ }
+
+ /**
+ * Is a statistic empty?
+ * @return true if the sample count is 0
+ */
+ @JsonIgnore
+ public boolean isEmpty() {
+ return samples == 0;
+ }
+
+ /**
+ * Set the values to 0.
+ */
+ public void clear() {
+ setSamplesAndSum(0, 0);
+ }
+
+ /**
+ * Set the sum and samples.
+ * Synchronized.
+ * @param sampleCount new sample count.
+ * @param newSum new sum
+ */
+ public synchronized void setSamplesAndSum(long sampleCount,
+ long newSum) {
+ setSamples(sampleCount);
+ setSum(newSum);
+ }
+
+ /**
+ * Set the statistic to the values of another.
+ * Synchronized.
+ * @param other the source.
+ */
+ public void set(final MeanStatistic other) {
+ setSamplesAndSum(other.getSamples(), other.getSum());
+ }
+
+ /**
+ * Set the sum.
+ * @param sum new sum
+ */
+ public synchronized void setSum(final long sum) {
+ this.sum = sum;
+ }
+
+ /**
+ * Set the sample count.
+ *
+ * If this is less than zero, it is set to zero.
+ *
+ * This stops an ill-formed JSON entry from
+ * breaking deserialization, or get an invalid sample count
+ * into an entry.
+ * @param samples sample count.
+ */
+ public synchronized void setSamples(final long samples) {
+ if (samples < 0) {
+ this.samples = 0;
+ } else {
+ this.samples = samples;
+ }
+ }
+
+ /**
+ * Get the arithmetic mean value.
+ *
+ * Thread safe.
+ * @return the mean
+ */
+ public synchronized double mean() {
+ return samples > 0
+ ? ((double) sum) / samples
+ : 0.0d;
+ }
+
+ /**
+ * Add another MeanStatistic.
+ * @param other other value
+ */
+ public synchronized MeanStatistic add(final MeanStatistic other) {
+ if (other.isEmpty()) {
+ return this;
+ }
+ if (isEmpty()) {
+ samples = other.samples;
+ sum = other.sum;
+ return this;
+ }
+ samples += other.samples;
+ sum += other.sum;
+ return this;
+ }
+
+ /**
+ * Add a sample.
+ *
+ * Thread safe.
+ * @param value value to add to the sum
+ */
+ public synchronized void addSample(long value) {
+ samples++;
+ sum += value;
+ }
+
+ /**
+ * The hash code is derived from the mean
+ * and sample count: if either is changed
+ * the statistic cannot be used as a key
+ * for hash tables/maps.
+ * @return a hash value
+ */
+ @Override
+ public synchronized int hashCode() {
+ return Objects.hash(sum, samples);
+ }
+
+ @Override
+ public synchronized boolean equals(final Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ MeanStatistic that = (MeanStatistic) o;
+ if (isEmpty()) {
+ // if we are empty, then so must the other.
+ return that.isEmpty();
+ }
+ return sum == that.sum &&
+ samples == that.samples;
+ }
+
+ @Override
+ public MeanStatistic clone() {
+ return copy();
+ }
+
+ /**
+ * Create a copy of this instance.
+ * @return copy.
+ *
+ */
+ public MeanStatistic copy() {
+ return new MeanStatistic(this);
+ }
+
+ @Override
+ public String toString() {
+ return String.format("(sum=%d, samples=%d, mean=%.4f)",
+ sum, samples, mean());
+ }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java
new file mode 100644
index 0000000000000..6c7572944accc
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java
@@ -0,0 +1,278 @@
+/*
+ * 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.statistics;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * These are common statistic names.
+ *
+ * When adding new common statistic name constants, please make them unique.
+ * By convention, they are implicitly unique:
+ *
+ * - the name of the constants are uppercase, words separated by
+ * underscores.
+ * - the value of the constants are lowercase of the constant names.
+ *
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class StoreStatisticNames {
+
+ /** {@value}. */
+ public static final String OP_APPEND = "op_append";
+
+ /** {@value}. */
+ public static final String OP_COPY_FROM_LOCAL_FILE =
+ "op_copy_from_local_file";
+
+ /** {@value}. */
+ public static final String OP_CREATE = "op_create";
+
+ /** {@value}. */
+ public static final String OP_CREATE_NON_RECURSIVE =
+ "op_create_non_recursive";
+
+ /** {@value}. */
+ public static final String OP_DELETE = "op_delete";
+
+ /** {@value}. */
+ public static final String OP_EXISTS = "op_exists";
+
+ /** {@value}. */
+ public static final String OP_GET_CONTENT_SUMMARY =
+ "op_get_content_summary";
+
+ /** {@value}. */
+ public static final String OP_GET_DELEGATION_TOKEN =
+ "op_get_delegation_token";
+
+ /** {@value}. */
+ public static final String OP_GET_FILE_CHECKSUM =
+ "op_get_file_checksum";
+
+ /** {@value}. */
+ public static final String OP_GET_FILE_STATUS = "op_get_file_status";
+
+ /** {@value}. */
+ public static final String OP_GET_STATUS = "op_get_status";
+
+ /** {@value}. */
+ public static final String OP_GLOB_STATUS = "op_glob_status";
+
+ /** {@value}. */
+ public static final String OP_IS_FILE = "op_is_file";
+
+ /** {@value}. */
+ public static final String OP_IS_DIRECTORY = "op_is_directory";
+
+ /** {@value}. */
+ public static final String OP_LIST_FILES = "op_list_files";
+
+ /** {@value}. */
+ public static final String OP_LIST_LOCATED_STATUS =
+ "op_list_located_status";
+
+ /** {@value}. */
+ public static final String OP_LIST_STATUS = "op_list_status";
+
+ /** {@value}. */
+ public static final String OP_MKDIRS = "op_mkdirs";
+
+ /** {@value}. */
+ public static final String OP_MODIFY_ACL_ENTRIES = "op_modify_acl_entries";
+
+ /** {@value}. */
+ public static final String OP_OPEN = "op_open";
+
+ /** {@value}. */
+ public static final String OP_REMOVE_ACL = "op_remove_acl";
+
+ /** {@value}. */
+ public static final String OP_REMOVE_ACL_ENTRIES = "op_remove_acl_entries";
+
+ /** {@value}. */
+ public static final String OP_REMOVE_DEFAULT_ACL = "op_remove_default_acl";
+
+ /** {@value}. */
+ public static final String OP_RENAME = "op_rename";
+
+ /** {@value}. */
+ public static final String OP_SET_ACL = "op_set_acl";
+
+ /** {@value}. */
+ public static final String OP_SET_OWNER = "op_set_owner";
+
+ /** {@value}. */
+ public static final String OP_SET_PERMISSION = "op_set_permission";
+
+ /** {@value}. */
+ public static final String OP_SET_TIMES = "op_set_times";
+
+ /** {@value}. */
+ public static final String OP_TRUNCATE = "op_truncate";
+
+ /** {@value}. */
+ public static final String DELEGATION_TOKENS_ISSUED
+ = "delegation_tokens_issued";
+
+ /** Requests throttled and retried: {@value}. */
+ public static final String STORE_IO_THROTTLED
+ = "store_io_throttled";
+
+ /** Requests made of a store: {@value}. */
+ public static final String STORE_IO_REQUEST
+ = "store_io_request";
+
+ /**
+ * IO retried: {@value}.
+ */
+ public static final String STORE_IO_RETRY
+ = "store_io_retry";
+
+ /**
+ * A store's equivalent of a paged LIST request was initiated: {@value}.
+ */
+ public static final String OBJECT_LIST_REQUEST
+ = "object_list_request";
+
+ /**
+ * Number of continued object listings made.
+ * Value :{@value}.
+ */
+ public static final String OBJECT_CONTINUE_LIST_REQUEST =
+ "object_continue_list_request";
+
+ /**
+ * A store's equivalent of a DELETE request was made: {@value}.
+ * This may be an HTTP DELETE verb, or it may be some custom
+ * operation which takes a list of objects to delete.
+ */
+ public static final String OP_HTTP_DELETE_REQUEST
+ = "op_http_delete_request";
+
+ /**
+ * Object multipart upload initiated.
+ * Value :{@value}.
+ */
+ public static final String OBJECT_MULTIPART_UPLOAD_INITIATED =
+ "object_multipart_initiated";
+
+ /**
+ * Object multipart upload aborted.
+ * Value :{@value}.
+ */
+ public static final String OBJECT_MULTIPART_UPLOAD_ABORTED =
+ "object_multipart_aborted";
+
+ /**
+ * Object put/multipart upload count.
+ * Value :{@value}.
+ */
+ public static final String OBJECT_PUT_REQUEST =
+ "object_put_request";
+
+ /**
+ * Object put/multipart upload completed count.
+ * Value :{@value}.
+ */
+ public static final String OBJECT_PUT_REQUEST_COMPLETED =
+ "object_put_request_completed";
+
+ /**
+ * Current number of active put requests.
+ * Value :{@value}.
+ */
+ public static final String OBJECT_PUT_REQUEST_ACTIVE =
+ "object_put_request_active";
+
+ /**
+ * number of bytes uploaded.
+ * Value :{@value}.
+ */
+ public static final String OBJECT_PUT_BYTES =
+ "object_put_bytes";
+
+ /**
+ * number of bytes queued for upload/being actively uploaded.
+ * Value :{@value}.
+ */
+ public static final String OBJECT_PUT_BYTES_PENDING =
+ "object_put_bytes_pending";
+
+ /**
+ * Count of S3 Select (or similar) requests issued.
+ * Value :{@value}.
+ */
+ public static final String OBJECT_SELECT_REQUESTS =
+ "object_select_requests";
+
+ /**
+ * Suffix to use for a minimum value when
+ * the same key is shared across min/mean/max
+ * statistics.
+ *
+ * Value {@value}.
+ */
+ public static final String SUFFIX_MIN = ".min";
+
+ /**
+ * Suffix to use for a maximum value when
+ * the same key is shared across max/mean/max
+ * statistics.
+ *
+ * Value {@value}.
+ */
+ public static final String SUFFIX_MAX = ".max";
+
+ /**
+ * Suffix to use for a mean statistic value when
+ * the same key is shared across mean/mean/max
+ * statistics.
+ *
+ * Value {@value}.
+ */
+ public static final String SUFFIX_MEAN = ".mean";
+
+
+ /**
+ * The name of the statistic collected for executor aquisition if
+ * a duration tracker factory is passed in to the constructor.
+ * {@value}.
+ */
+ public static String ACTION_EXECUTOR_ACQUIRED = "action_executor_acquired";
+
+ /**
+ * An HTTP HEAD request was made: {@value}.
+ */
+ public static final String ACTION_HTTP_HEAD_REQUEST
+ = "action_http_head_request";
+
+ /**
+ * An HTTP GET request was made: {@value}.
+ */
+ public static final String ACTION_HTTP_GET_REQUEST
+ = "action_http_get_request";
+
+
+ private StoreStatisticNames() {
+ }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.java
new file mode 100644
index 0000000000000..37da830512497
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.java
@@ -0,0 +1,188 @@
+/*
+ * 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.statistics;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * These are common statistic names.
+ *
+ * When adding new common statistic name constants, please make them unique.
+ * By convention, they are implicitly unique:
+ *
+ * -
+ * The name of the constants are uppercase, words separated by
+ * underscores.
+ *
+ * -
+ * The value of the constants are lowercase of the constant names.
+ *
+ *
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class StreamStatisticNames {
+
+ /** {@value}. */
+ public static final String STREAM_READ_ABORTED = "stream_aborted";
+
+ /** {@value}. */
+ public static final String STREAM_READ_BYTES_DISCARDED_ABORT
+ = "stream_read_bytes_discarded_in_abort";
+
+ /** {@value}. */
+ public static final String STREAM_READ_BYTES_DISCARDED_SEEK
+ = "stream_read_bytes_discarded_in_seek";
+
+ /** {@value}. */
+ public static final String STREAM_READ_CLOSED = "stream_read_closed";
+
+ /** {@value}. */
+ public static final String STREAM_READ_CLOSE_BYTES_READ
+ = "stream_read_bytes_read_in_close";
+
+ /** {@value}. */
+ public static final String STREAM_READ_CLOSE_OPERATIONS
+ = "stream_read_close_operations";
+
+ /** {@value}. */
+ public static final String STREAM_READ_OPENED = "stream_read_opened";
+
+ /** {@value}. */
+ public static final String STREAM_READ_BYTES
+ = "stream_read_bytes";
+
+ /** {@value}. */
+ public static final String STREAM_READ_EXCEPTIONS =
+ "stream_read_exceptions";
+
+ /** {@value}. */
+ public static final String STREAM_READ_FULLY_OPERATIONS
+ = "stream_read_fully_operations";
+
+ /** {@value}. */
+ public static final String STREAM_READ_OPERATIONS =
+ "stream_read_operations";
+
+ /** {@value}. */
+ public static final String STREAM_READ_OPERATIONS_INCOMPLETE
+ = "stream_read_operations_incomplete";
+
+ /** {@value}. */
+ public static final String STREAM_READ_VERSION_MISMATCHES
+ = "stream_read_version_mismatches";
+
+ /** {@value}. */
+ public static final String STREAM_READ_SEEK_BYTES_BACKWARDS
+ = "stream_read_bytes_backwards_on_seek";
+
+ /** {@value}. */
+ public static final String STREAM_READ_SEEK_FORWARD_OPERATIONS
+ = "stream_read_seek_forward_operations";
+
+ /** {@value}. */
+ public static final String STREAM_READ_SEEK_POLICY_CHANGED =
+ "stream_read_seek_policy_changed";
+
+ /** {@value}. */
+ public static final String STREAM_READ_SEEK_OPERATIONS =
+ "stream_read_seek_operations";
+
+ /** {@value}. */
+ public static final String STREAM_READ_SEEK_BACKWARD_OPERATIONS =
+ "stream_read_seek_backward_operations";
+
+ /** {@value}. */
+ public static final String STREAM_READ_SEEK_BYTES_READ =
+ "stream_read_seek_bytes_read";
+
+ /** {@value}. */
+ public static final String STREAM_READ_SEEK_BYTES_SKIPPED
+ = "stream_read_bytes_skipped_on_seek";
+
+ /** {@value}. */
+ public static final String STREAM_READ_SKIP_OPERATIONS =
+ "stream_read_skip_operations";
+
+ /** {@value}. */
+ public static final String STREAM_READ_SKIP_BYTES =
+ "stream_read_skip_bytes";
+
+ /** {@value}. */
+ public static final String STREAM_READ_TOTAL_BYTES
+ = "stream_read_total_bytes";
+
+ /** {@value}. */
+ public static final String STREAM_WRITE_EXCEPTIONS =
+ "stream_write_exceptions";
+
+ /** Failures when finalizing a multipart upload: {@value}. */
+ public static final String STREAM_WRITE_EXCEPTIONS_COMPLETING_UPLOADS =
+ "stream_write_exceptions_completing_upload";
+
+ /** {@value}. */
+ public static final String STREAM_WRITE_BLOCK_UPLOADS
+ = "stream_write_block_uploads";
+
+ /** {@value}. */
+ public static final String STREAM_WRITE_BLOCK_UPLOADS_ABORTED
+ = "stream_write_block_uploads_aborted";
+
+ /** {@value}. */
+ public static final String STREAM_WRITE_BLOCK_UPLOADS_ACTIVE
+ = "stream_write_block_uploads_active";
+
+ /** {@value}. */
+ public static final String STREAM_WRITE_BLOCK_UPLOADS_COMMITTED
+ = "stream_write_block_uploads_committed";
+
+ /** {@value}. */
+ public static final String STREAM_WRITE_BLOCK_UPLOADS_PENDING
+ = "stream_write_block_uploads_pending";
+
+ /** {@value}. */
+ public static final String STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING =
+ "stream_write_block_uploads_data_pending";
+
+ /**
+ * "Count of bytes written to output stream including all not yet uploaded.
+ * {@value}.
+ */
+ public static final String STREAM_WRITE_BYTES
+ = "stream_write_bytes";
+
+ /**
+ * Count of total time taken for uploads to complete.
+ * {@value}.
+ */
+ public static final String STREAM_WRITE_TOTAL_TIME
+ = "stream_write_total_time";
+
+ /**
+ * Total queue duration of all block uploads.
+ * {@value}.
+ */
+ public static final String STREAM_WRITE_QUEUE_DURATION
+ = "stream_write_queue_duration";
+
+ private StreamStatisticNames() {
+ }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/AbstractIOStatisticsImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/AbstractIOStatisticsImpl.java
new file mode 100644
index 0000000000000..c701a509d8951
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/AbstractIOStatisticsImpl.java
@@ -0,0 +1,30 @@
+/*
+ * 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.statistics.impl;
+
+import org.apache.hadoop.fs.statistics.IOStatistics;
+
+/**
+ * Base implementation in case common methods/fields need to be added
+ * in future.
+ */
+public abstract class AbstractIOStatisticsImpl implements IOStatistics {
+
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsInputStream.java
new file mode 100644
index 0000000000000..718487a7d7242
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsInputStream.java
@@ -0,0 +1,74 @@
+/*
+ * 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.statistics.impl;
+
+import java.io.BufferedInputStream;
+import java.io.InputStream;
+
+import org.apache.hadoop.fs.StreamCapabilities;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+
+import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics;
+
+/**
+ * An extension of {@code BufferedInputStream} which implements
+ * {@link IOStatisticsSource} and forwards requests for the
+ * {@link IOStatistics} to the wrapped stream.
+ *
+ * This should be used when any input stream needs buffering while
+ * allowing the inner stream to be a source of statistics.
+ *
+ * It also implements {@link StreamCapabilities} and forwards the probe
+ * to the inner stream, if possible.
+ */
+public class BufferedIOStatisticsInputStream
+ extends BufferedInputStream
+ implements IOStatisticsSource, StreamCapabilities {
+
+ public BufferedIOStatisticsInputStream(final InputStream in) {
+ super(in);
+ }
+
+ public BufferedIOStatisticsInputStream(final InputStream in, final int size) {
+ super(in, size);
+ }
+
+ @Override
+ public IOStatistics getIOStatistics() {
+ return retrieveIOStatistics(in);
+ }
+
+ /**
+ * If the inner stream supports {@link StreamCapabilities},
+ * forward the probe to it.
+ * Otherwise: return false.
+ *
+ * @param capability string to query the stream support for.
+ * @return true if a capability is known to be supported.
+ */
+ @Override
+ public boolean hasCapability(final String capability) {
+ if (in instanceof StreamCapabilities) {
+ return ((StreamCapabilities) in).hasCapability(capability);
+ } else {
+ return false;
+ }
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsOutputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsOutputStream.java
new file mode 100644
index 0000000000000..849f008328b0b
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsOutputStream.java
@@ -0,0 +1,152 @@
+/*
+ * 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.statistics.impl;
+
+import java.io.BufferedOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+
+import org.apache.hadoop.fs.StreamCapabilities;
+import org.apache.hadoop.fs.Syncable;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+
+import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics;
+
+/**
+ * An extension of {@code BufferedOutputStream} which implements
+ * {@link IOStatisticsSource} and forwards requests for the
+ * {@link IOStatistics} to the wrapped stream.
+ *
+ * This should be used when any output stream needs buffering while
+ * allowing the inner stream to be a source of statistics.
+ *
+ * It also implements {@link StreamCapabilities}
+ * and {@link Syncable} and forwards to to the inner stream,
+ * if possible.
+ */
+public class BufferedIOStatisticsOutputStream
+ extends BufferedOutputStream
+ implements IOStatisticsSource, Syncable, StreamCapabilities {
+
+ /**
+ * Should calls to Syncable downgrade to flush if the underlying
+ * stream does not support it?
+ * While that breaks a core contract requirement of Syncable:
+ * "Sync.sync() guarantees durability", downgrading is
+ * the default behavior of FsDataOutputStream.
+ */
+ private final boolean downgradeSyncable;
+
+ /**
+ * Construct with default buffer size.
+ * @param out output stream to buffer
+ * @param downgradeSyncable should Syncable calls downgrade?
+ */
+ public BufferedIOStatisticsOutputStream(final OutputStream out,
+ final boolean downgradeSyncable) {
+ super(out);
+ this.downgradeSyncable = downgradeSyncable;
+ }
+
+ /**
+ * Construct with custom buffer size.
+ *
+ * @param out output stream to buffer
+ * @param size buffer.
+ * @param downgradeSyncable should Syncable calls downgrade?
+ */
+ public BufferedIOStatisticsOutputStream(final OutputStream out,
+ final int size, final boolean downgradeSyncable) {
+ super(out, size);
+ this.downgradeSyncable = downgradeSyncable;
+ }
+
+ @Override
+ public IOStatistics getIOStatistics() {
+ return retrieveIOStatistics(out);
+ }
+
+ /**
+ * If the inner stream supports {@link StreamCapabilities},
+ * forward the probe to it.
+ * Otherwise: return false.
+ *
+ * @param capability string to query the stream support for.
+ * @return true if a capability is known to be supported.
+ */
+ @Override
+ public boolean hasCapability(final String capability) {
+ if (out instanceof StreamCapabilities) {
+ return ((StreamCapabilities) out).hasCapability(capability);
+ } else {
+ return false;
+ }
+ }
+
+ /**
+ * If the inner stream is Syncable, flush the buffer and then
+ * invoke the inner stream's hflush() operation.
+ *
+ * Otherwise: throw an exception, unless the stream was constructed with
+ * {@link #downgradeSyncable} set to true, in which case the stream
+ * is just flushed.
+ * @throws IOException IO Problem
+ * @throws UnsupportedOperationException if the inner class is not syncable
+ */
+ @Override
+ public void hflush() throws IOException {
+ if (out instanceof Syncable) {
+ flush();
+ ((Syncable) out).hflush();
+ } else {
+ if (!downgradeSyncable) {
+ throw new UnsupportedOperationException("hflush not supported by "
+ + out);
+ } else {
+ flush();
+ }
+ }
+ }
+
+ /**
+ * If the inner stream is Syncable, flush the buffer and then
+ * invoke the inner stream's hsync() operation.
+ *
+ * Otherwise: throw an exception, unless the stream was constructed with
+ * {@link #downgradeSyncable} set to true, in which case the stream
+ * is just flushed.
+ * @throws IOException IO Problem
+ * @throws UnsupportedOperationException if the inner class is not syncable
+ */
+ @Override
+ public void hsync() throws IOException {
+ if (out instanceof Syncable) {
+ flush();
+ ((Syncable) out).hsync();
+ } else {
+ if (!downgradeSyncable) {
+ throw new UnsupportedOperationException("hsync not supported by "
+ + out);
+ } else {
+ flush();
+ }
+ }
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/DynamicIOStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/DynamicIOStatistics.java
new file mode 100644
index 0000000000000..22bc2e77dab2c
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/DynamicIOStatistics.java
@@ -0,0 +1,132 @@
+/*
+ * 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.statistics.impl;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.function.Function;
+
+import org.apache.hadoop.fs.statistics.MeanStatistic;
+
+/**
+ * These statistics are dynamically evaluated by the supplied
+ * String -> type functions.
+ *
+ * This allows statistic sources to supply a list of callbacks used to
+ * generate the statistics on demand; similar to some of the Coda Hale metrics.
+ *
+ * The evaluation actually takes place during the iteration's {@code next()}
+ * call.
+ */
+final class DynamicIOStatistics
+ extends AbstractIOStatisticsImpl {
+
+ /**
+ * Counter evaluators.
+ */
+ private final EvaluatingStatisticsMap counters
+ = new EvaluatingStatisticsMap<>("counters");
+
+ private final EvaluatingStatisticsMap gauges
+ = new EvaluatingStatisticsMap<>("gauges");
+
+ private final EvaluatingStatisticsMap minimums
+ = new EvaluatingStatisticsMap<>("minimums");
+
+ private final EvaluatingStatisticsMap maximums
+ = new EvaluatingStatisticsMap<>("maximums");
+
+ private final EvaluatingStatisticsMap meanStatistics
+ = new EvaluatingStatisticsMap<>("meanStatistics", MeanStatistic::copy);
+
+ DynamicIOStatistics() {
+ }
+
+ @Override
+ public Map counters() {
+ return Collections.unmodifiableMap(counters);
+ }
+
+ @Override
+ public Map gauges() {
+ return Collections.unmodifiableMap(gauges);
+ }
+
+ @Override
+ public Map minimums() {
+ return Collections.unmodifiableMap(minimums);
+ }
+
+ @Override
+ public Map maximums() {
+ return Collections.unmodifiableMap(maximums);
+ }
+
+ @Override
+ public Map meanStatistics() {
+ return Collections.unmodifiableMap(meanStatistics);
+ }
+
+ /**
+ * add a mapping of a key to a counter function.
+ * @param key the key
+ * @param eval the evaluator
+ */
+ void addCounterFunction(String key, Function eval) {
+ counters.addFunction(key, eval);
+ }
+
+ /**
+ * add a mapping of a key to a gauge function.
+ * @param key the key
+ * @param eval the evaluator
+ */
+ void addGaugeFunction(String key, Function eval) {
+ gauges.addFunction(key, eval);
+ }
+
+ /**
+ * add a mapping of a key to a minimum function.
+ * @param key the key
+ * @param eval the evaluator
+ */
+ void addMinimumFunction(String key, Function eval) {
+ minimums.addFunction(key, eval);
+ }
+
+ /**
+ * add a mapping of a key to a maximum function.
+ * @param key the key
+ * @param eval the evaluator
+ */
+ void addMaximumFunction(String key, Function eval) {
+ maximums.addFunction(key, eval);
+ }
+
+ /**
+ * add a mapping of a key to a meanStatistic function.
+ * @param key the key
+ * @param eval the evaluator
+ */
+ void addMeanStatisticFunction(String key,
+ Function eval) {
+ meanStatistics.addFunction(key, eval);
+ }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/DynamicIOStatisticsBuilder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/DynamicIOStatisticsBuilder.java
new file mode 100644
index 0000000000000..0150048229e44
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/DynamicIOStatisticsBuilder.java
@@ -0,0 +1,248 @@
+/*
+ * 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.statistics.impl;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Function;
+import java.util.function.ToLongFunction;
+
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.MeanStatistic;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+
+import static com.google.common.base.Preconditions.checkState;
+
+/**
+ * Builder of {@link DynamicIOStatistics}.
+ *
+ * Instantiate through
+ * {@link IOStatisticsBinding#dynamicIOStatistics()}.
+ */
+public class DynamicIOStatisticsBuilder {
+
+ /**
+ * the instance being built up. Will be null after the (single)
+ * call to {@link #build()}.
+ */
+ private DynamicIOStatistics instance = new DynamicIOStatistics();
+
+ /**
+ * Build the IOStatistics instance.
+ * @return an instance.
+ * @throws IllegalStateException if the builder has already been built.
+ */
+ public IOStatistics build() {
+ final DynamicIOStatistics stats = activeInstance();
+ // stop the builder from working any more.
+ instance = null;
+ return stats;
+ }
+
+ /**
+ * Get the statistics instance.
+ * @return the instance to build/return
+ * @throws IllegalStateException if the builder has already been built.
+ */
+ private DynamicIOStatistics activeInstance() {
+ checkState(instance != null, "Already built");
+ return instance;
+ }
+
+ /**
+ * Add a new evaluator to the counter statistics.
+ * @param key key of this statistic
+ * @param eval evaluator for the statistic
+ * @return the builder.
+ */
+ public DynamicIOStatisticsBuilder withLongFunctionCounter(String key,
+ ToLongFunction eval) {
+ activeInstance().addCounterFunction(key, eval::applyAsLong);
+ return this;
+ }
+
+ /**
+ * Add a counter statistic to dynamically return the
+ * latest value of the source.
+ * @param key key of this statistic
+ * @param source atomic long counter
+ * @return the builder.
+ */
+ public DynamicIOStatisticsBuilder withAtomicLongCounter(String key,
+ AtomicLong source) {
+ withLongFunctionCounter(key, s -> source.get());
+ return this;
+ }
+
+ /**
+ * Add a counter statistic to dynamically return the
+ * latest value of the source.
+ * @param key key of this statistic
+ * @param source atomic int counter
+ * @return the builder.
+ */
+ public DynamicIOStatisticsBuilder withAtomicIntegerCounter(String key,
+ AtomicInteger source) {
+ withLongFunctionCounter(key, s -> source.get());
+ return this;
+ }
+
+ /**
+ * Build a dynamic counter statistic from a
+ * {@link MutableCounterLong}.
+ * @param key key of this statistic
+ * @param source mutable long counter
+ * @return the builder.
+ */
+ public DynamicIOStatisticsBuilder withMutableCounter(String key,
+ MutableCounterLong source) {
+ withLongFunctionCounter(key, s -> source.value());
+ return this;
+ }
+
+ /**
+ * Add a new evaluator to the gauge statistics.
+ * @param key key of this statistic
+ * @param eval evaluator for the statistic
+ * @return the builder.
+ */
+ public DynamicIOStatisticsBuilder withLongFunctionGauge(String key,
+ ToLongFunction eval) {
+ activeInstance().addGaugeFunction(key, eval::applyAsLong);
+ return this;
+ }
+
+ /**
+ * Add a gauge statistic to dynamically return the
+ * latest value of the source.
+ * @param key key of this statistic
+ * @param source atomic long gauge
+ * @return the builder.
+ */
+ public DynamicIOStatisticsBuilder withAtomicLongGauge(String key,
+ AtomicLong source) {
+ withLongFunctionGauge(key, s -> source.get());
+ return this;
+ }
+
+ /**
+ * Add a gauge statistic to dynamically return the
+ * latest value of the source.
+ * @param key key of this statistic
+ * @param source atomic int gauge
+ * @return the builder.
+ */
+ public DynamicIOStatisticsBuilder withAtomicIntegerGauge(String key,
+ AtomicInteger source) {
+ withLongFunctionGauge(key, s -> source.get());
+ return this;
+ }
+
+ /**
+ * Add a new evaluator to the minimum statistics.
+ * @param key key of this statistic
+ * @param eval evaluator for the statistic
+ * @return the builder.
+ */
+ public DynamicIOStatisticsBuilder withLongFunctionMinimum(String key,
+ ToLongFunction eval) {
+ activeInstance().addMinimumFunction(key, eval::applyAsLong);
+ return this;
+ }
+
+ /**
+ * Add a minimum statistic to dynamically return the
+ * latest value of the source.
+ * @param key key of this statistic
+ * @param source atomic long minimum
+ * @return the builder.
+ */
+ public DynamicIOStatisticsBuilder withAtomicLongMinimum(String key,
+ AtomicLong source) {
+ withLongFunctionMinimum(key, s -> source.get());
+ return this;
+ }
+
+ /**
+ * Add a minimum statistic to dynamically return the
+ * latest value of the source.
+ * @param key key of this statistic
+ * @param source atomic int minimum
+ * @return the builder.
+ */
+ public DynamicIOStatisticsBuilder withAtomicIntegerMinimum(String key,
+ AtomicInteger source) {
+ withLongFunctionMinimum(key, s -> source.get());
+ return this;
+ }
+
+
+ /**
+ * Add a new evaluator to the maximum statistics.
+ * @param key key of this statistic
+ * @param eval evaluator for the statistic
+ * @return the builder.
+ */
+ public DynamicIOStatisticsBuilder withLongFunctionMaximum(String key,
+ ToLongFunction eval) {
+ activeInstance().addMaximumFunction(key, eval::applyAsLong);
+ return this;
+ }
+
+ /**
+ * Add a maximum statistic to dynamically return the
+ * latest value of the source.
+ * @param key key of this statistic
+ * @param source atomic long maximum
+ * @return the builder.
+ */
+ public DynamicIOStatisticsBuilder withAtomicLongMaximum(String key,
+ AtomicLong source) {
+ withLongFunctionMaximum(key, s -> source.get());
+ return this;
+ }
+
+ /**
+ * Add a maximum statistic to dynamically return the
+ * latest value of the source.
+ * @param key key of this statistic
+ * @param source atomic int maximum
+ * @return the builder.
+ */
+ public DynamicIOStatisticsBuilder withAtomicIntegerMaximum(String key,
+ AtomicInteger source) {
+ withLongFunctionMaximum(key, s -> source.get());
+ return this;
+ }
+
+ /**
+ * Add a new evaluator to the mean statistics.
+ *
+ * This is a function which must return the mean and the sample count.
+ * @param key key of this statistic
+ * @param eval evaluator for the statistic
+ * @return the builder.
+ */
+ public DynamicIOStatisticsBuilder withMeanStatisticFunction(String key,
+ Function eval) {
+ activeInstance().addMeanStatisticFunction(key, eval);
+ return this;
+ }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EmptyIOStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EmptyIOStatistics.java
new file mode 100644
index 0000000000000..f474fc209771c
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EmptyIOStatistics.java
@@ -0,0 +1,74 @@
+/*
+ * 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.statistics.impl;
+
+import java.util.Map;
+
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.MeanStatistic;
+
+import static java.util.Collections.emptyMap;
+
+/**
+ * An empty IO Statistics implementation for classes which always
+ * want to return a non-null set of statistics.
+ */
+final class EmptyIOStatistics extends AbstractIOStatisticsImpl {
+
+ /**
+ * The sole instance of this class.
+ */
+ private static final EmptyIOStatistics INSTANCE = new EmptyIOStatistics();
+
+ private EmptyIOStatistics() {
+ }
+
+ @Override
+ public Map counters() {
+ return emptyMap();
+ }
+
+ @Override
+ public Map gauges() {
+ return emptyMap();
+ }
+
+ @Override
+ public Map minimums() {
+ return emptyMap();
+ }
+
+ @Override
+ public Map maximums() {
+ return emptyMap();
+ }
+
+ @Override
+ public Map meanStatistics() {
+ return emptyMap();
+ }
+
+ /**
+ * Get the single instance of this class.
+ * @return a shared, empty instance.
+ */
+ public static IOStatistics getInstance() {
+ return INSTANCE;
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EvaluatingStatisticsMap.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EvaluatingStatisticsMap.java
new file mode 100644
index 0000000000000..1249d0c843c21
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EvaluatingStatisticsMap.java
@@ -0,0 +1,205 @@
+/*
+ * 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.statistics.impl;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * A map of functions which can be invoked to dynamically
+ * create the value of an entry.
+ * @param type of entry value.
+ */
+final class EvaluatingStatisticsMap implements
+ Map {
+
+ /**
+ * Functions to invoke when evaluating keys.
+ */
+ private final Map> evaluators
+ = new ConcurrentHashMap<>();
+
+ /**
+ * Function to use when copying map values.
+ */
+ private final Function copyFn;
+
+ /**
+ * Name for use in getter/error messages.
+ */
+ private final String name;
+
+ /**
+ * Construct with the copy function being simple passthrough.
+ * @param name map name
+ */
+ EvaluatingStatisticsMap(final String name) {
+ this(name, IOStatisticsBinding::passthroughFn);
+ }
+
+ /**
+ * Construct with the copy function being that supplied in.
+ * @param name map name
+ * @param copyFn copy function.
+ */
+ EvaluatingStatisticsMap(final String name,
+ final Function copyFn) {
+ this.name = name;
+ this.copyFn = copyFn;
+ }
+
+ /**
+ * add a mapping of a key to a function.
+ * @param key the key
+ * @param eval the evaluator
+ */
+ void addFunction(String key, Function eval) {
+ evaluators.put(key, eval);
+ }
+
+ @Override
+ public int size() {
+ return evaluators.size();
+ }
+
+ @Override
+ public boolean isEmpty() {
+ return evaluators.isEmpty();
+ }
+
+ @Override
+ public boolean containsKey(final Object key) {
+ return evaluators.containsKey(key);
+ }
+
+ @Override
+ public boolean containsValue(final Object value) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public E get(final Object key) {
+ Function fn = evaluators.get(key);
+ return fn != null
+ ? fn.apply((String) key)
+ : null;
+ }
+
+ @Override
+ public E put(final String key, final E value) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public E remove(final Object key) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void putAll(final Map extends String, ? extends E> m) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void clear() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public Set keySet() {
+ return evaluators.keySet();
+ }
+
+ /**
+ * Takes a snapshot and then provide an iterator around this.
+ * @return the iterator.
+ */
+ @Override
+ public Collection values() {
+ return snapshot().values();
+ }
+
+ /**
+ * Take a snapshot.
+ * @return a map snapshot.
+ */
+ public TreeMap snapshot() {
+ return IOStatisticsBinding.snapshotMap(this, copyFn);
+ }
+
+ /**
+ * Creating the entry set forces an evaluation of the functions.
+ *
+ * The evaluation may be parallelized.
+ * @return an evaluated set of values
+ */
+ @Override
+ public synchronized Set> entrySet() {
+ Set>> evalEntries =
+ evaluators.entrySet();
+ Set> r = evalEntries.parallelStream().map((e) ->
+ new EntryImpl<>(e.getKey(), e.getValue().apply(e.getKey())))
+ .collect(Collectors.toSet());
+ return r;
+ }
+
+ String getName() {
+ return name;
+ }
+
+ /**
+ * Simple entry.
+ * @param entry type
+ */
+ private static final class EntryImpl implements Entry {
+
+ private String key;
+
+ private E value;
+
+ private EntryImpl(final String key, final E value) {
+ this.key = key;
+ this.value = value;
+ }
+
+ @Override
+ public String getKey() {
+ return key;
+ }
+
+ @Override
+ public E getValue() {
+ return value;
+ }
+
+ @Override
+ public E setValue(final E val) {
+ this.value = val;
+ return val;
+ }
+ }
+
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java
new file mode 100644
index 0000000000000..e3b1568d97636
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java
@@ -0,0 +1,336 @@
+/*
+ * 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.statistics.impl;
+
+import java.io.Serializable;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.hadoop.fs.StorageStatistics;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.apache.hadoop.fs.statistics.MeanStatistic;
+
+import static org.apache.hadoop.fs.statistics.IOStatistics.MIN_UNSET_VALUE;
+
+/**
+ * Support for implementing IOStatistics interfaces.
+ */
+public final class IOStatisticsBinding {
+
+ /** Pattern used for each entry. */
+ public static final String ENTRY_PATTERN = "(%s=%s)";
+
+ /** String to return when a source is null. */
+ @VisibleForTesting
+ public static final String NULL_SOURCE = "()";
+
+ private IOStatisticsBinding() {
+ }
+
+ /**
+ * Create IOStatistics from a storage statistics instance.
+ *
+ * This will be updated as the storage statistics change.
+ * @param storageStatistics source data.
+ * @return an IO statistics source.
+ */
+ public static IOStatistics fromStorageStatistics(
+ StorageStatistics storageStatistics) {
+ DynamicIOStatisticsBuilder builder = dynamicIOStatistics();
+ Iterator it = storageStatistics
+ .getLongStatistics();
+ while (it.hasNext()) {
+ StorageStatistics.LongStatistic next = it.next();
+ builder.withLongFunctionCounter(next.getName(),
+ k -> storageStatistics.getLong(k));
+ }
+ return builder.build();
+ }
+
+ /**
+ * Create a builder for dynamic IO Statistics.
+ * @return a builder to be completed.
+ */
+ public static DynamicIOStatisticsBuilder dynamicIOStatistics() {
+ return new DynamicIOStatisticsBuilder();
+ }
+
+ /**
+ * Get the shared instance of the immutable empty statistics
+ * object.
+ * @return an empty statistics object.
+ */
+ public static IOStatistics emptyStatistics() {
+ return EmptyIOStatistics.getInstance();
+ }
+
+ /**
+ * Take an IOStatistics instance and wrap it in a source.
+ * @param statistics statistics.
+ * @return a source which will return the values
+ */
+ public static IOStatisticsSource wrap(IOStatistics statistics) {
+ return new SourceWrappedStatistics(statistics);
+ }
+
+ /**
+ * Create a builder for an {@link IOStatisticsStore}.
+ *
+ * @return a builder instance.
+ */
+ public static IOStatisticsStoreBuilder iostatisticsStore() {
+ return new IOStatisticsStoreBuilderImpl();
+ }
+
+ /**
+ * Convert an entry to the string format used in logging.
+ *
+ * @param entry entry to evaluate
+ * @param entry type
+ * @return formatted string
+ */
+ public static String entryToString(
+ final Map.Entry entry) {
+ return entryToString(entry.getKey(), entry.getValue());
+ }
+
+ /**
+ * Convert entry values to the string format used in logging.
+ *
+ * @param name statistic name
+ * @param value stat value
+ * @return formatted string
+ */
+ public static String entryToString(
+ final String name, final E value) {
+ return String.format(
+ ENTRY_PATTERN,
+ name,
+ value);
+ }
+
+ /**
+ * Copy into the dest map all the source entries.
+ * The destination is cleared first.
+ * @param entry type
+ * @param dest destination of the copy
+ * @param source source
+ * @param copyFn function to copy entries
+ * @return the destination.
+ */
+ private static Map copyMap(
+ Map dest,
+ Map source,
+ Function copyFn) {
+ // we have to clone the values so that they aren't
+ // bound to the original values
+ dest.clear();
+ source.entrySet()
+ .forEach(entry ->
+ dest.put(entry.getKey(), copyFn.apply(entry.getValue())));
+ return dest;
+ }
+
+ /**
+ * A passthrough copy operation suitable for immutable
+ * types, including numbers.
+ * @param src source object
+ * @return the source object
+ */
+ public static E passthroughFn(E src) {
+ return src;
+ }
+
+ /**
+ * Take a snapshot of a supplied map, where the copy option simply
+ * uses the existing value.
+ *
+ * For this to be safe, the map must refer to immutable objects.
+ * @param source source map
+ * @param type of values.
+ * @return a new map referencing the same values.
+ */
+ public static TreeMap snapshotMap(
+ Map source) {
+ return snapshotMap(source,
+ IOStatisticsBinding::passthroughFn);
+ }
+
+ /**
+ * Take a snapshot of a supplied map, using the copy function
+ * to replicate the source values.
+ * @param source source map
+ * @param copyFn function to copy the value
+ * @param type of values.
+ * @return a new map referencing the same values.
+ */
+ public static TreeMap snapshotMap(
+ Map source,
+ Function copyFn) {
+ TreeMap dest = new TreeMap<>();
+ copyMap(dest, source, copyFn);
+ return dest;
+ }
+
+ /**
+ * Aggregate two maps so that the destination.
+ * @param type of values
+ * @param dest destination map.
+ * @param other other map
+ * @param aggregateFn function to aggregate the values.
+ * @param copyFn function to copy the value
+ */
+ public static void aggregateMaps(
+ Map dest,
+ Map other,
+ BiFunction aggregateFn,
+ final Function copyFn) {
+ // scan through the other hand map; copy
+ // any values not in the left map,
+ // aggregate those for which there is already
+ // an entry
+ other.entrySet().forEach(entry -> {
+ String key = entry.getKey();
+ E rVal = entry.getValue();
+ E lVal = dest.get(key);
+ if (lVal == null) {
+ dest.put(key, copyFn.apply(rVal));
+ } else {
+ dest.put(key, aggregateFn.apply(lVal, rVal));
+ }
+ });
+ }
+
+ /**
+ * Aggregate two counters.
+ * @param l left value
+ * @param r right value
+ * @return the aggregate value
+ */
+ public static Long aggregateCounters(Long l, Long r) {
+ return Math.max(l, 0) + Math.max(r, 0);
+ }
+
+ /**
+ * Add two gauges.
+ * @param l left value
+ * @param r right value
+ * @return aggregate value
+ */
+ public static Long aggregateGauges(Long l, Long r) {
+ return l + r;
+ }
+
+
+ /**
+ * Aggregate two minimum values.
+ * @param l left
+ * @param r right
+ * @return the new minimum.
+ */
+ public static Long aggregateMinimums(Long l, Long r) {
+ if (l == MIN_UNSET_VALUE) {
+ return r;
+ } else if (r == MIN_UNSET_VALUE) {
+ return l;
+ } else {
+ return Math.min(l, r);
+ }
+ }
+
+ /**
+ * Aggregate two maximum values.
+ * @param l left
+ * @param r right
+ * @return the new minimum.
+ */
+ public static Long aggregateMaximums(Long l, Long r) {
+ if (l == MIN_UNSET_VALUE) {
+ return r;
+ } else if (r == MIN_UNSET_VALUE) {
+ return l;
+ } else {
+ return Math.max(l, r);
+ }
+ }
+
+ /**
+ * Aggregate the mean statistics.
+ * This returns a new instance.
+ * @param l left value
+ * @param r right value
+ * @return aggregate value
+ */
+ public static MeanStatistic aggregateMeanStatistics(
+ MeanStatistic l, MeanStatistic r) {
+ MeanStatistic res = l.copy();
+ res.add(r);
+ return res;
+ }
+
+ /**
+ * Update a maximum value tracked in an atomic long.
+ * This is thread safe -it uses compareAndSet to ensure
+ * that Thread T1 whose sample is greater than the current
+ * value never overwrites an update from thread T2 whose
+ * sample was also higher -and which completed first.
+ * @param dest destination for all changes.
+ * @param sample sample to update.
+ */
+ public static void maybeUpdateMaximum(AtomicLong dest, long sample) {
+ boolean done;
+ do {
+ long current = dest.get();
+ if (sample > current) {
+ done = dest.compareAndSet(current, sample);
+ } else {
+ done = true;
+ }
+ } while (!done);
+ }
+
+ /**
+ * Update a maximum value tracked in an atomic long.
+ * This is thread safe -it uses compareAndSet to ensure
+ * that Thread T1 whose sample is greater than the current
+ * value never overwrites an update from thread T2 whose
+ * sample was also higher -and which completed first.
+ * @param dest destination for all changes.
+ * @param sample sample to update.
+ */
+ public static void maybeUpdateMinimum(AtomicLong dest, long sample) {
+ boolean done;
+ do {
+ long current = dest.get();
+ if (current == MIN_UNSET_VALUE || sample < current) {
+ done = dest.compareAndSet(current, sample);
+ } else {
+ done = true;
+ }
+ } while (!done);
+ }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStore.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStore.java
new file mode 100644
index 0000000000000..08bf7e929af9c
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStore.java
@@ -0,0 +1,259 @@
+/*
+ * 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.statistics.impl;
+
+import java.time.Duration;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsAggregator;
+import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
+import org.apache.hadoop.fs.statistics.MeanStatistic;
+
+/**
+ * Interface of an IOStatistics store intended for
+ * use in classes which track statistics for reporting.
+ */
+public interface IOStatisticsStore extends IOStatistics,
+ IOStatisticsAggregator,
+ DurationTrackerFactory {
+
+ /**
+ * Increment a counter by one.
+ *
+ * No-op if the counter is unknown.
+ * @param key statistics key
+ * @return old value or, if the counter is unknown: 0
+ */
+ default long incrementCounter(String key) {
+ return incrementCounter(key, 1);
+ }
+
+ /**
+ * Increment a counter.
+ *
+ * No-op if the counter is unknown.
+ * @param key statistics key
+ * @param value value to increment
+ * @return old value or, if the counter is unknown: 0
+ */
+ long incrementCounter(String key, long value);
+
+ /**
+ * Set a counter.
+ *
+ * No-op if the counter is unknown.
+ * @param key statistics key
+ * @param value value to set
+ */
+ void setCounter(String key, long value);
+
+ /**
+ * Set a gauge.
+ *
+ * No-op if the gauge is unknown.
+ * @param key statistics key
+ * @param value value to set
+ */
+ void setGauge(String key, long value);
+
+ /**
+ * Increment a gauge.
+ *
+ * No-op if the gauge is unknown.
+ * @param key statistics key
+ * @param value value to increment
+ * @return old value or 0
+ */
+ long incrementGauge(String key, long value);
+
+ /**
+ * Set a maximum.
+ * No-op if the maximum is unknown.
+ * @param key statistics key
+ * @param value value to set
+ */
+ void setMaximum(String key, long value);
+
+ /**
+ * Increment a maximum.
+ *
+ * No-op if the maximum is unknown.
+ * @param key statistics key
+ * @param value value to increment
+ * @return old value or 0
+ */
+ long incrementMaximum(String key, long value);
+
+ /**
+ * Set a minimum.
+ *
+ * No-op if the minimum is unknown.
+ * @param key statistics key
+ * @param value value to set
+ */
+ void setMinimum(String key, long value);
+
+ /**
+ * Increment a minimum.
+ *
+ * No-op if the minimum is unknown.
+ * @param key statistics key
+ * @param value value to increment
+ * @return old value or 0
+ */
+ long incrementMinimum(String key, long value);
+
+ /**
+ * Add a minimum sample: if less than the current value,
+ * updates the value.
+ *
+ * No-op if the minimum is unknown.
+ * @param key statistics key
+ * @param value sample value
+ */
+ void addMinimumSample(String key, long value);
+
+ /**
+ * Add a maximum sample: if greater than the current value,
+ * updates the value.
+ *
+ * No-op if the maximum is unknown.
+ * @param key statistics key
+ * @param value sample value
+ */
+ void addMaximumSample(String key, long value);
+
+
+ /**
+ * Set a mean statistic to a given value.
+ *
+ * No-op if the maximum is unknown.
+ * @param key statistic key
+ * @param value new value.
+ */
+ void setMeanStatistic(String key, MeanStatistic value);
+
+ /**
+ * Add a sample to the mean statistics.
+ *
+ * No-op if the maximum is unknown.
+ * @param key key
+ * @param value sample value.
+ */
+ void addMeanStatisticSample(String key, long value);
+
+ /**
+ * Reset all statistics.
+ * Unsynchronized.
+ */
+ void reset();
+
+ /**
+ * Update the counter values from a statistics source.
+ *
+ * The source must have all keys in this instance;
+ * extra keys are ignored.
+ * @param source source of statistics.
+ */
+ void copy(IOStatistics source);
+
+ /**
+ * Get a reference to the atomic instance providing the
+ * value for a specific counter. This is useful if
+ * the value is passed around.
+ * @param key statistic name
+ * @return the reference
+ * @throws NullPointerException if there is no entry of that name
+ */
+ AtomicLong getCounterReference(String key);
+
+ /**
+ * Get a reference to the atomic instance providing the
+ * value for a specific maximum. This is useful if
+ * the value is passed around.
+ * @param key statistic name
+ * @return the reference
+ * @throws NullPointerException if there is no entry of that name
+ */
+ AtomicLong getMaximumReference(String key);
+
+ /**
+ * Get a reference to the atomic instance providing the
+ * value for a specific minimum. This is useful if
+ * the value is passed around.
+ * @param key statistic name
+ * @return the reference
+ * @throws NullPointerException if there is no entry of that name
+ */
+ AtomicLong getMinimumReference(String key);
+
+ /**
+ * Get a reference to the atomic instance providing the
+ * value for a specific gauge. This is useful if
+ * the value is passed around.
+ * @param key statistic name
+ * @return the reference
+ * @throws NullPointerException if there is no entry of that name
+ */
+ AtomicLong getGaugeReference(String key);
+
+ /**
+ * Get a reference to the atomic instance providing the
+ * value for a specific meanStatistic. This is useful if
+ * the value is passed around.
+ * @param key statistic name
+ * @return the reference
+ * @throws NullPointerException if there is no entry of that name
+ */
+ MeanStatistic getMeanStatistic(String key);
+
+ /**
+ * Add a duration to the min/mean/max statistics, using the
+ * given prefix and adding a suffix for each specific value.
+ *
+ * The update is not-atomic, even though each individual statistic
+ * is updated thread-safely. If two threads update the values
+ * simultaneously, at the end of each operation the state will
+ * be correct. It is only during the sequence that the statistics
+ * may be observably inconsistent.
+ * @param prefix statistic prefix
+ * @param durationMillis duration in milliseconds.
+ */
+ void addTimedOperation(String prefix, long durationMillis);
+
+ /**
+ * Add a duration to the min/mean/max statistics, using the
+ * given prefix and adding a suffix for each specific value.;
+ * increment tha counter whose name == prefix.
+ *
+ * If any of the statistics are not registered, that part of
+ * the sequence will be omitted -the rest will proceed.
+ *
+ * The update is not-atomic, even though each individual statistic
+ * is updated thread-safely. If two threads update the values
+ * simultaneously, at the end of each operation the state will
+ * be correct. It is only during the sequence that the statistics
+ * may be observably inconsistent.
+ * @param prefix statistic prefix
+ * @param duration duration
+ */
+ void addTimedOperation(String prefix, Duration duration);
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStoreBuilder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStoreBuilder.java
new file mode 100644
index 0000000000000..d94a8389b7ff8
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStoreBuilder.java
@@ -0,0 +1,75 @@
+/*
+ * 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.statistics.impl;
+
+/**
+ * Builder of the {@link IOStatisticsStore} implementation.
+ */
+public interface IOStatisticsStoreBuilder {
+
+ /**
+ * Declare a varargs list of counters to add.
+ * @param keys names of statistics.
+ * @return this builder.
+ */
+ IOStatisticsStoreBuilder withCounters(String... keys);
+
+ /**
+ * Declare a varargs list of gauges to add.
+ * @param keys names of statistics.
+ * @return this builder.
+ */
+ IOStatisticsStoreBuilder withGauges(String... keys);
+
+ /**
+ * Declare a varargs list of maximums to add.
+ * @param keys names of statistics.
+ * @return this builder.
+ */
+ IOStatisticsStoreBuilder withMaximums(String... keys);
+
+ /**
+ * Declare a varargs list of minimums to add.
+ * @param keys names of statistics.
+ * @return this builder.
+ */
+ IOStatisticsStoreBuilder withMinimums(String... keys);
+
+ /**
+ * Declare a varargs list of means to add.
+ * @param keys names of statistics.
+ * @return this builder.
+ */
+ IOStatisticsStoreBuilder withMeanStatistics(String... keys);
+
+ /**
+ * Add a statistic in the counter, min, max and mean maps for each
+ * declared statistic prefix.
+ * @param prefixes prefixes for the stats.
+ * @return this
+ */
+ IOStatisticsStoreBuilder withDurationTracking(
+ String... prefixes);
+
+ /**
+ * Build the collector.
+ * @return a new collector.
+ */
+ IOStatisticsStore build();
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStoreBuilderImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStoreBuilderImpl.java
new file mode 100644
index 0000000000000..5068b24043a54
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStoreBuilderImpl.java
@@ -0,0 +1,93 @@
+/*
+ * 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.statistics.impl;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MAX;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MEAN;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MIN;
+
+/**
+ * Builder implementation.
+ */
+final class IOStatisticsStoreBuilderImpl implements
+ IOStatisticsStoreBuilder {
+
+ private final List counters = new ArrayList<>();
+
+ private final List gauges = new ArrayList<>();
+
+ private final List minimums = new ArrayList<>();
+
+ private final List maximums = new ArrayList<>();
+
+ private final List meanStatistics = new ArrayList<>();
+
+ @Override
+ public IOStatisticsStoreBuilderImpl withCounters(final String... keys) {
+ counters.addAll(Arrays.asList(keys));
+ return this;
+ }
+
+ @Override
+ public IOStatisticsStoreBuilderImpl withGauges(final String... keys) {
+ gauges.addAll(Arrays.asList(keys));
+ return this;
+ }
+
+ @Override
+ public IOStatisticsStoreBuilderImpl withMaximums(final String... keys) {
+ maximums.addAll(Arrays.asList(keys));
+ return this;
+ }
+
+ @Override
+ public IOStatisticsStoreBuilderImpl withMinimums(final String... keys) {
+ minimums.addAll(Arrays.asList(keys));
+ return this;
+ }
+
+ @Override
+ public IOStatisticsStoreBuilderImpl withMeanStatistics(
+ final String... keys) {
+ meanStatistics.addAll(Arrays.asList(keys));
+ return this;
+ }
+
+ @Override
+ public IOStatisticsStoreBuilderImpl withDurationTracking(
+ final String... prefixes) {
+ for (String p : prefixes) {
+ withCounters(p);
+ withMinimums(p + SUFFIX_MIN);
+ withMaximums(p + SUFFIX_MAX);
+ withMeanStatistics(p + SUFFIX_MEAN);
+ }
+ return this;
+ }
+
+ @Override
+ public IOStatisticsStore build() {
+ return new IOStatisticsStoreImpl(counters, gauges, minimums,
+ maximums, meanStatistics);
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStoreImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStoreImpl.java
new file mode 100644
index 0000000000000..285c3647a341a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStoreImpl.java
@@ -0,0 +1,421 @@
+/*
+ * 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.statistics.impl;
+
+import javax.annotation.Nullable;
+import java.time.Duration;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.statistics.DurationTracker;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.MeanStatistic;
+
+import static java.util.Objects.requireNonNull;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MAX;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MEAN;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MIN;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.aggregateMaximums;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.aggregateMinimums;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.dynamicIOStatistics;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.maybeUpdateMaximum;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.maybeUpdateMinimum;
+
+/**
+ * Implementation of {@link IOStatisticsStore}.
+ */
+final class IOStatisticsStoreImpl extends WrappedIOStatistics
+ implements IOStatisticsStore {
+
+ /**
+ * Log changes at debug.
+ * Noisy, but occasionally useful.
+ */
+ private static final Logger LOG =
+ LoggerFactory.getLogger(IOStatisticsStoreImpl.class);
+
+ private final Map counterMap = new HashMap<>();
+
+ private final Map gaugeMap = new HashMap<>();
+
+ private final Map minimumMap = new HashMap<>();
+
+ private final Map maximumMap = new HashMap<>();
+
+ private final Map meanStatisticMap
+ = new HashMap<>();
+
+ /**
+ * Constructor invoked via the builder.
+ * @param counters keys to use for the counter statistics.
+ * @param gauges names of gauges
+ * @param minimums names of minimums
+ * @param maximums names of maximums
+ * @param meanStatistics names of mean statistics.
+ */
+ IOStatisticsStoreImpl(
+ final List counters,
+ final List gauges,
+ final List minimums,
+ final List maximums,
+ final List meanStatistics) {
+ // initially create the superclass with no wrapped mapping;
+ super(null);
+
+ // now construct a dynamic statistics source mapping to
+ // the various counters, gauges etc dynamically created
+ // into maps
+ DynamicIOStatisticsBuilder builder = dynamicIOStatistics();
+ if (counters != null) {
+ for (String key : counters) {
+ AtomicLong counter = new AtomicLong();
+ counterMap.put(key, counter);
+ builder.withAtomicLongCounter(key, counter);
+ }
+ }
+ if (gauges != null) {
+ for (String key : gauges) {
+ AtomicLong gauge = new AtomicLong();
+ gaugeMap.put(key, gauge);
+ builder.withAtomicLongGauge(key, gauge);
+ }
+ }
+ if (maximums != null) {
+ for (String key : maximums) {
+ AtomicLong maximum = new AtomicLong(MAX_UNSET_VALUE);
+ maximumMap.put(key, maximum);
+ builder.withAtomicLongMaximum(key, maximum);
+ }
+ }
+ if (minimums != null) {
+ for (String key : minimums) {
+ AtomicLong minimum = new AtomicLong(MIN_UNSET_VALUE);
+ minimumMap.put(key, minimum);
+ builder.withAtomicLongMinimum(key, minimum);
+ }
+ }
+ if (meanStatistics != null) {
+ for (String key : meanStatistics) {
+ meanStatisticMap.put(key, new MeanStatistic());
+ builder.withMeanStatisticFunction(key, k -> {
+ return meanStatisticMap.get(k);
+ });
+ }
+ }
+ setWrapped(builder.build());
+ }
+
+ /**
+ * Set an atomic long to a value.
+ * @param aLong atomic long; may be null
+ * @param value value to set to
+ */
+ private void setAtomicLong(final AtomicLong aLong, final long value) {
+ if (aLong != null) {
+ aLong.set(value);
+ }
+ }
+
+ /**
+ * increment an atomic long and return its value;
+ * null long is no-op returning 0.
+ * @param aLong atomic long; may be null
+ * @param increment amount to increment; -ve for a decrement
+ * @return final value or 0
+ */
+ private long incAtomicLong(final AtomicLong aLong,
+ final long increment) {
+ if (aLong != null) {
+ return aLong.addAndGet(increment);
+ } else {
+ return 0;
+ }
+ }
+
+ @Override
+ public void setCounter(final String key, final long value) {
+ setAtomicLong(counterMap.get(key), value);
+ LOG.debug("Setting counter {} to {}", key, value);
+
+ }
+
+ @Override
+ public long incrementCounter(final String key, final long value) {
+ long l = incAtomicLong(counterMap.get(key), value);
+ LOG.debug("Incrementing counter {} by {} with final value {}",
+ key, value, l);
+ return l;
+ }
+
+ @Override
+ public void setMaximum(final String key, final long value) {
+ setAtomicLong(maximumMap.get(key), value);
+ }
+
+ @Override
+ public long incrementMaximum(final String key, final long value) {
+ return incAtomicLong(maximumMap.get(key), value);
+ }
+
+ @Override
+ public void setMinimum(final String key, final long value) {
+ setAtomicLong(minimumMap.get(key), value);
+ }
+
+ @Override
+ public long incrementMinimum(final String key, final long value) {
+ return incAtomicLong(minimumMap.get(key), value);
+ }
+
+ @Override
+ public void addMinimumSample(final String key, final long value) {
+ AtomicLong min = minimumMap.get(key);
+ if (min != null) {
+ maybeUpdateMinimum(min, value);
+ }
+ }
+
+ @Override
+ public void addMaximumSample(final String key, final long value) {
+ AtomicLong max = maximumMap.get(key);
+ if (max != null) {
+ maybeUpdateMaximum(max, value);
+ }
+ }
+
+ @Override
+ public void setGauge(final String key, final long value) {
+ setAtomicLong(gaugeMap.get(key), value);
+ }
+
+ @Override
+ public long incrementGauge(final String key, final long value) {
+ return incAtomicLong(gaugeMap.get(key), value);
+ }
+
+ @Override
+ public void setMeanStatistic(final String key, final MeanStatistic value) {
+ final MeanStatistic ref = meanStatisticMap.get(key);
+ if (ref != null) {
+ ref.setSamplesAndSum(value.getSamples(), value.getSum());
+ }
+ }
+
+ @Override
+ public void addMeanStatisticSample(final String key, final long value) {
+ final MeanStatistic ref = meanStatisticMap.get(key);
+ if (ref != null) {
+ ref.addSample(value);
+ }
+ }
+
+ /**
+ * Reset all statistics.
+ * Unsynchronized.
+ */
+ @Override
+ public void reset() {
+ counterMap.values().forEach(a -> a.set(0));
+ gaugeMap.values().forEach(a -> a.set(0));
+ minimumMap.values().forEach(a -> a.set(0));
+ maximumMap.values().forEach(a -> a.set(0));
+ meanStatisticMap.values().forEach(a -> a.clear());
+ }
+
+ @Override
+ public void copy(final IOStatistics source) {
+ counterMap.entrySet().forEach(e -> {
+ e.getValue().set(lookup(source.counters(), e.getKey()));
+ });
+ gaugeMap.entrySet().forEach(e -> {
+ e.getValue().set(lookup(source.gauges(), e.getKey()));
+ });
+ maximumMap.entrySet().forEach(e -> {
+ e.getValue().set(lookup(source.maximums(), e.getKey()));
+ });
+ minimumMap.entrySet().forEach(e -> {
+ e.getValue().set(lookup(source.minimums(), e.getKey()));
+ });
+ meanStatisticMap.entrySet().forEach(e -> {
+ String key = e.getKey();
+ MeanStatistic statisticValue = lookup(source.meanStatistics(), key);
+ e.getValue().set(statisticValue);
+ });
+ }
+
+ /**
+ * Aggregate those statistics which the store is tracking;
+ * ignore the rest.
+ *
+ * @param statistics statistics; may be null
+ * @return true if a statistics reference was supplied/aggregated.
+ */
+ @Override
+ public boolean aggregate(@Nullable final IOStatistics statistics) {
+ if (statistics == null) {
+ return false;
+ }
+ // counters: addition
+ counterMap.entrySet().forEach(e -> {
+ e.getValue().addAndGet(lookup(statistics.counters(), e.getKey()));
+ });
+ // gauge: add positive values only
+ gaugeMap.entrySet().forEach(e -> {
+ long sourceGauge = lookup(statistics.gauges(), e.getKey());
+ if (sourceGauge > 0) {
+ e.getValue().addAndGet(sourceGauge);
+ }
+ });
+ // min: min of current and source
+ minimumMap.entrySet().forEach(e -> {
+ AtomicLong dest = e.getValue();
+ long sourceValue = lookup(statistics.minimums(), e.getKey());
+ dest.set(aggregateMinimums(dest.get(), sourceValue));
+ });
+ // max: max of current and source
+ maximumMap.entrySet().forEach(e -> {
+ AtomicLong dest = e.getValue();
+ long sourceValue = lookup(statistics.maximums(), e.getKey());
+ dest.set(aggregateMaximums(dest.get(), sourceValue));
+ });
+ // the most complex
+ meanStatisticMap.entrySet().forEach(e -> {
+ MeanStatistic current = e.getValue();
+ MeanStatistic sourceValue = lookup(
+ statistics.meanStatistics(), e.getKey());
+ current.add(sourceValue);
+ });
+ return true;
+ }
+
+ /**
+ * Get a reference to the map type providing the
+ * value for a specific key, raising an exception if
+ * there is no entry for that key.
+ * @param type of map/return type.
+ * @param map map to look up
+ * @param key statistic name
+ * @return the value
+ * @throws NullPointerException if there is no entry of that name
+ */
+ private static T lookup(final Map map, String key) {
+ T val = map.get(key);
+ requireNonNull(val, () -> ("unknown statistic " + key));
+ return val;
+ }
+
+ /**
+ * Get a reference to the atomic instance providing the
+ * value for a specific counter. This is useful if
+ * the value is passed around.
+ * @param key statistic name
+ * @return the reference
+ * @throws NullPointerException if there is no entry of that name
+ */
+ @Override
+ public AtomicLong getCounterReference(String key) {
+ return lookup(counterMap, key);
+ }
+
+ /**
+ * Get a reference to the atomic instance providing the
+ * value for a specific maximum. This is useful if
+ * the value is passed around.
+ * @param key statistic name
+ * @return the reference
+ * @throws NullPointerException if there is no entry of that name
+ */
+ @Override
+ public AtomicLong getMaximumReference(String key) {
+ return lookup(maximumMap, key);
+ }
+
+ /**
+ * Get a reference to the atomic instance providing the
+ * value for a specific minimum. This is useful if
+ * the value is passed around.
+ * @param key statistic name
+ * @return the reference
+ * @throws NullPointerException if there is no entry of that name
+ */
+ @Override
+ public AtomicLong getMinimumReference(String key) {
+ return lookup(minimumMap, key);
+ }
+
+ /**
+ * Get a reference to the atomic instance providing the
+ * value for a specific gauge. This is useful if
+ * the value is passed around.
+ * @param key statistic name
+ * @return the reference
+ * @throws NullPointerException if there is no entry of that name
+ */
+ @Override
+ public AtomicLong getGaugeReference(String key) {
+ return lookup(gaugeMap, key);
+ }
+
+ /**
+ * Get a reference to the atomic instance providing the
+ * value for a specific meanStatistic. This is useful if
+ * the value is passed around.
+ * @param key statistic name
+ * @return the reference
+ * @throws NullPointerException if there is no entry of that name
+ */
+ @Override
+ public MeanStatistic getMeanStatistic(String key) {
+ return lookup(meanStatisticMap, key);
+ }
+
+ /**
+ * Add a duration to the min/mean/max statistics, using the
+ * given prefix and adding a suffix for each specific value.
+ *
+ * The update is not-atomic, even though each individual statistic
+ * is updated thread-safely. If two threads update the values
+ * simultaneously, at the end of each operation the state will
+ * be correct. It is only during the sequence that the statistics
+ * may be observably inconsistent.
+ * @param prefix statistic prefix
+ * @param durationMillis duration in milliseconds.
+ */
+ @Override
+ public void addTimedOperation(String prefix, long durationMillis) {
+ addMeanStatisticSample(prefix + SUFFIX_MEAN, durationMillis);
+ addMinimumSample(prefix + SUFFIX_MIN, durationMillis);
+ addMaximumSample(prefix + SUFFIX_MAX, durationMillis);
+ }
+
+ @Override
+ public void addTimedOperation(String prefix, Duration duration) {
+ addTimedOperation(prefix, duration.toMillis());
+ }
+
+ @Override
+ public DurationTracker trackDuration(final String key, final int count) {
+ return new StatisticDurationTracker(this, key, count);
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/SourceWrappedStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/SourceWrappedStatistics.java
new file mode 100644
index 0000000000000..5aced7c5cddbf
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/SourceWrappedStatistics.java
@@ -0,0 +1,44 @@
+/*
+ * 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.statistics.impl;
+
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+
+/**
+ * Wrap a statistics instance with an {@link IOStatisticsSource}
+ * instance which will then serve up the statistics when asked.
+ */
+public class SourceWrappedStatistics implements IOStatisticsSource {
+
+ private final IOStatistics source;
+
+ /**
+ * Constructor.
+ * @param source source of statistics.
+ */
+ public SourceWrappedStatistics(final IOStatistics source) {
+ this.source = source;
+ }
+
+ @Override
+ public IOStatistics getIOStatistics() {
+ return source;
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StatisticDurationTracker.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StatisticDurationTracker.java
new file mode 100644
index 0000000000000..ba2c9015d6311
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StatisticDurationTracker.java
@@ -0,0 +1,83 @@
+/*
+ * 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.statistics.impl;
+
+import org.apache.hadoop.fs.statistics.DurationTracker;
+import org.apache.hadoop.util.OperationDuration;
+
+/**
+ * Track the duration of an object.
+ *
+ * When closed the
+ * min/max/mean statistics are updated.
+ *
+ * In the constructor, the counter with name of 'key' is
+ * incremented -default is by 1, but can be set to other
+ * values, including 0.
+ */
+public class StatisticDurationTracker extends OperationDuration
+ implements DurationTracker {
+
+ /**
+ * statistics to update.
+ */
+ private final IOStatisticsStore iostats;
+
+ /**
+ * Key to use as prefix of values.
+ */
+ private final String key;
+
+ /**
+ * Constructor -increments the counter by 1.
+ * @param iostats statistics to update
+ * @param key prefix of values.
+ */
+ public StatisticDurationTracker(final IOStatisticsStore iostats,
+ final String key) {
+ this(iostats, key, 1);
+ }
+
+ /**
+ * Constructor.
+ * If the supplied count is greater than xero, the counter
+ * of the key name is updated.
+ * @param iostats statistics to update
+ * @param key Key to use as prefix of values.
+ * @param count #of times to increment the matching counter.
+ */
+ public StatisticDurationTracker(final IOStatisticsStore iostats,
+ final String key,
+ final int count) {
+ this.iostats = iostats;
+ this.key = key;
+ if (count > 0) {
+ iostats.incrementCounter(key, count);
+ }
+ }
+
+ /**
+ * Set the finished time and then update the statistics.
+ */
+ @Override
+ public void close() {
+ finished();
+ iostats.addTimedOperation(key, asDuration());
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StubDurationTracker.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StubDurationTracker.java
new file mode 100644
index 0000000000000..87cc77877f4bd
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StubDurationTracker.java
@@ -0,0 +1,39 @@
+/*
+ * 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.statistics.impl;
+
+import org.apache.hadoop.fs.statistics.DurationTracker;
+
+/**
+ * A simple stub duration tracker which can be issued in interfaces
+ * and other places where full duration tracking is not implemented.
+ */
+public final class StubDurationTracker implements DurationTracker {
+
+ public static final DurationTracker STUB_DURATION_TRACKER =
+ new StubDurationTracker();
+
+ private StubDurationTracker() {
+ }
+
+ @Override
+ public void close() {
+
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StubDurationTrackerFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StubDurationTrackerFactory.java
new file mode 100644
index 0000000000000..3bb13d2e0473a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StubDurationTrackerFactory.java
@@ -0,0 +1,43 @@
+/*
+ * 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.statistics.impl;
+
+import org.apache.hadoop.fs.statistics.DurationTracker;
+import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
+
+/**
+ * This is a stub factory which always returns no-op duration
+ * trackers. Allows for code to always be handed a factory.
+ */
+public final class StubDurationTrackerFactory implements DurationTrackerFactory {
+
+ /**
+ * Single instance.
+ */
+ public static final StubDurationTrackerFactory STUB_DURATION_TRACKER_FACTORY
+ = new StubDurationTrackerFactory();
+
+ private StubDurationTrackerFactory() {
+ }
+
+ @Override
+ public DurationTracker trackDuration(final String key, final int count) {
+ return StubDurationTracker.STUB_DURATION_TRACKER;
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/WrappedIOStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/WrappedIOStatistics.java
new file mode 100644
index 0000000000000..6617ba40edac4
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/WrappedIOStatistics.java
@@ -0,0 +1,108 @@
+/*
+ * 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.statistics.impl;
+
+import java.util.Map;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.MeanStatistic;
+
+import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToString;
+
+/**
+ * Wrap IOStatistics source with another (dynamic) wrapper.
+ */
+public class WrappedIOStatistics extends AbstractIOStatisticsImpl {
+
+ /**
+ * The wrapped statistics.
+ */
+ private IOStatistics wrapped;
+
+ /**
+ * Instantiate.
+ * @param wrapped nullable wrapped statistics.
+ */
+ public WrappedIOStatistics(final IOStatistics wrapped) {
+ this.wrapped = wrapped;
+ }
+
+ /**
+ * Instantiate without setting the statistics.
+ * This is for subclasses which build up the map during their own
+ * construction.
+ */
+ protected WrappedIOStatistics() {
+ }
+
+ @Override
+ public Map counters() {
+ return getWrapped().counters();
+ }
+
+ /**
+ * Get at the wrapped inner statistics.
+ * @return the wrapped value
+ */
+ protected IOStatistics getWrapped() {
+ return wrapped;
+ }
+
+ /**
+ * Set the wrapped statistics.
+ * Will fail if the field is already set.
+ * @param wrapped new value
+ */
+ protected void setWrapped(final IOStatistics wrapped) {
+ Preconditions.checkState(this.wrapped == null,
+ "Attempted to overwrite existing wrapped statistics");
+ this.wrapped = wrapped;
+ }
+
+ @Override
+ public Map gauges() {
+ return getWrapped().gauges();
+ }
+
+ @Override
+ public Map minimums() {
+ return getWrapped().minimums();
+ }
+
+ @Override
+ public Map maximums() {
+ return getWrapped().maximums();
+ }
+
+ @Override
+ public Map meanStatistics() {
+ return getWrapped().meanStatistics();
+ }
+
+ /**
+ * Return the statistics dump of the wrapped statistics.
+ * @return the statistics for logging.
+ */
+ @Override
+ public String toString() {
+ return ioStatisticsToString(wrapped);
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/package-info.java
new file mode 100644
index 0000000000000..3ff7dacadce7a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/package-info.java
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+/**
+ * Implementation support for statistics.
+ * For use internally; external filesystems MAY use this if the implementors
+ * accept that it is unstable and that incompatible changes may take
+ * place over minor point releases.
+ */
+
+@InterfaceAudience.LimitedPrivate("Filesystems")
+@InterfaceStability.Unstable
+package org.apache.hadoop.fs.statistics.impl;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/package-info.java
new file mode 100644
index 0000000000000..fafa775163f79
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/package-info.java
@@ -0,0 +1,133 @@
+/*
+ * 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.
+ */
+
+/**
+ * This package contains support for statistic collection and reporting.
+ * This is the public API; implementation classes are to be kept elsewhere.
+ *
+ * This package defines two interfaces:
+ *
+ * {@link org.apache.hadoop.fs.statistics.IOStatisticsSource}:
+ * a source of statistic data, which can be retrieved
+ * through a call to
+ * {@link org.apache.hadoop.fs.statistics.IOStatisticsSource#getIOStatistics()} .
+ *
+ * {@link org.apache.hadoop.fs.statistics.IOStatistics} the statistics retrieved
+ * from a statistics source.
+ *
+ * The retrieved statistics may be an immutable snapshot -in which case to get
+ * updated statistics another call to
+ * {@link org.apache.hadoop.fs.statistics.IOStatisticsSource#getIOStatistics()}
+ * must be made. Or they may be dynamic -in which case every time a specific
+ * statistic is retrieved, the latest version is returned. Callers should assume
+ * that if a statistics instance is dynamic, there is no atomicity when querying
+ * multiple statistics. If the statistics source was a closeable object (e.g. a
+ * stream), the statistics MUST remain valid after the stream is closed.
+ *
+ * Use pattern:
+ *
+ * An application probes an object (filesystem, stream etc) to see if it
+ * implements {@code IOStatisticsSource}, and, if it is,
+ * calls {@code getIOStatistics()} to get its statistics.
+ * If this is non-null, the client has statistics on the current
+ * state of the statistics.
+ *
+ * The expectation is that a statistics source is dynamic: when a value is
+ * looked up the most recent values are returned.
+ * When iterating through the set, the values of the iterator SHOULD
+ * be frozen at the time the iterator was requested.
+ *
+ * These statistics can be used to: log operations, profile applications,
+ * and make assertions about the state of the output.
+ *
+ * The names of statistics are a matter of choice of the specific source.
+ * However, {@link org.apache.hadoop.fs.statistics.StoreStatisticNames}
+ * contains a
+ * set of names recommended for object store operations.
+ * {@link org.apache.hadoop.fs.statistics.StreamStatisticNames} declares
+ * recommended names for statistics provided for
+ * input and output streams.
+ *
+ * Utility classes are includes to assist use
+ *
+ * -
+ * {@link org.apache.hadoop.fs.statistics.IOStatisticsSupport}.
+ * General support, including the ability to take a serializable
+ * snapshot of the current state of an IOStatistics instance.
+ *
+ * -
+ * {@link org.apache.hadoop.fs.statistics.IOStatisticsLogging}.
+ * Methods for robust/on-demand string conversion, designed
+ * for use in logging statements and {@code toString()} implementations.
+ *
* -
+ * {@link org.apache.hadoop.fs.statistics.IOStatisticsSnapshot}.
+ * A static snaphot of statistics which can be marshalled via
+ * java serialization or as JSON via jackson. It supports
+ * aggregation, so can be used to generate aggregate statistics.
+ *
+ *
+ *
+ *
+ * Implementors notes:
+ *
+ * -
+ * IOStatistics keys SHOULD be standard names where possible.
+ *
+ * -
+ * An IOStatistics instance MUST be unique to that specific instance of
+ * {@link org.apache.hadoop.fs.statistics.IOStatisticsSource}.
+ * (i.e. not shared the way StorageStatistics are)
+ *
+ * -
+ * MUST return the same values irrespective of which thread the statistics are
+ * retrieved or its keys evaluated.
+ *
+ * -
+ * MUST NOT remove keys once a statistic instance has been created.
+ *
+ * -
+ * MUST NOT add keys once a statistic instance has been created.
+ *
+ * -
+ * MUST NOT block for long periods of time while blocking operations
+ * (reads, writes) are taking place in the source.
+ * That is: minimal synchronization points (AtomicLongs etc.) may be
+ * used to share values, but retrieval of statistics should
+ * be fast and return values even while slow/blocking remote IO is underway.
+ *
+ * -
+ * MUST support value enumeration and retrieval after the source has been
+ * closed.
+ *
+ * -
+ * SHOULD NOT have back-references to potentially expensive objects
+ * (filesystem instances etc.)
+ *
+ * -
+ * SHOULD provide statistics which can be added to generate aggregate
+ * statistics.
+ *
+ *
+ */
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+package org.apache.hadoop.fs.statistics;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionInputStream.java
index 2dfa30bf76ec4..55bb132e9c87c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionInputStream.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionInputStream.java
@@ -25,6 +25,10 @@
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.PositionedReadable;
import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.apache.hadoop.fs.statistics.IOStatisticsSupport;
+
/**
* A compression input stream.
*
@@ -34,7 +38,8 @@
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
-public abstract class CompressionInputStream extends InputStream implements Seekable {
+public abstract class CompressionInputStream extends InputStream
+ implements Seekable, IOStatisticsSource {
/**
* The input stream to be compressed.
*/
@@ -68,7 +73,16 @@ public void close() throws IOException {
}
}
}
-
+
+ /**
+ * Return any IOStatistics provided by the underlying stream.
+ * @return IO stats from the inner stream.
+ */
+ @Override
+ public IOStatistics getIOStatistics() {
+ return IOStatisticsSupport.retrieveIOStatistics(in);
+ }
+
/**
* Read bytes from the stream.
* Made abstract to prevent leakage to underlying stream.
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionOutputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionOutputStream.java
index 71c7f32e665e5..2a11ace81702c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionOutputStream.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionOutputStream.java
@@ -23,13 +23,17 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.apache.hadoop.fs.statistics.IOStatisticsSupport;
/**
* A compression output stream.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
-public abstract class CompressionOutputStream extends OutputStream {
+public abstract class CompressionOutputStream extends OutputStream
+ implements IOStatisticsSource {
/**
* The output stream to be compressed.
*/
@@ -94,4 +98,12 @@ public void flush() throws IOException {
*/
public abstract void resetState() throws IOException;
+ /**
+ * Return any IOStatistics provided by the underlying stream.
+ * @return IO stats from the inner stream.
+ */
+ @Override
+ public IOStatistics getIOStatistics() {
+ return IOStatisticsSupport.retrieveIOStatistics(out);
+ }
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java
index e2cd3048d5843..520ddf6bdf401 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java
@@ -25,6 +25,9 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.apache.hadoop.fs.statistics.IOStatisticsSupport;
import org.apache.hadoop.io.Text;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
@@ -42,7 +45,7 @@
*/
@InterfaceAudience.LimitedPrivate({"MapReduce"})
@InterfaceStability.Unstable
-public class LineReader implements Closeable {
+public class LineReader implements Closeable, IOStatisticsSource {
private static final int DEFAULT_BUFFER_SIZE = 64 * 1024;
private int bufferSize = DEFAULT_BUFFER_SIZE;
private InputStream in;
@@ -148,7 +151,16 @@ public LineReader(InputStream in, Configuration conf,
public void close() throws IOException {
in.close();
}
-
+
+ /**
+ * Return any IOStatistics provided by the source.
+ * @return IO stats from the input stream.
+ */
+ @Override
+ public IOStatistics getIOStatistics() {
+ return IOStatisticsSupport.retrieveIOStatistics(in);
+ }
+
/**
* Read one line from the InputStream into the given Text.
*
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/OperationDuration.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/OperationDuration.java
index 3276d2138bbfc..fdd25286a2300 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/OperationDuration.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/OperationDuration.java
@@ -18,48 +18,98 @@
package org.apache.hadoop.util;
+import java.time.Duration;
+
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Little duration counter.
*/
-@InterfaceAudience.Private
+@InterfaceAudience.Public
@InterfaceStability.Unstable
public class OperationDuration {
+ /**
+ * Time in millis when the operation started.
+ */
private final long started;
+
+ /**
+ * Time when the operation finished.
+ */
private long finished;
+ /**
+ * Instantiate.
+ * The start time and finished time are both set
+ * to the current clock time.
+ */
public OperationDuration() {
started = time();
finished = started;
}
+ /**
+ * Evaluate the system time.
+ * @return the current clock time.
+ */
protected long time() {
return System.currentTimeMillis();
}
+ /**
+ * Update the finished time with the current system time.
+ */
public void finished() {
finished = time();
}
+ /**
+ * Return the duration as {@link #humanTime(long)}.
+ * @return a printable duration.
+ */
public String getDurationString() {
return humanTime(value());
}
+ /**
+ * Convert to a human time of minutes:seconds.millis.
+ * @param time time to humanize.
+ * @return a printable value.
+ */
public static String humanTime(long time) {
long seconds = (time / 1000);
long minutes = (seconds / 60);
return String.format("%d:%02d.%03ds", minutes, seconds % 60, time % 1000);
}
+ /**
+ * Return the duration as {@link #humanTime(long)}.
+ * @return a printable duration.
+ */
@Override
public String toString() {
return getDurationString();
}
+ /**
+ * Get the duration in milliseconds.
+ *
+ * This will be 0 until a call
+ * to {@link #finished()} has been made.
+ * @return the currently recorded duration.
+ */
public long value() {
return finished -started;
}
+
+ /**
+ * Get the duration of an operation as a java Duration
+ * instance.
+ * @return a duration.
+ */
+ public Duration asDuration() {
+ return Duration.ofMillis(value());
+ }
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SemaphoredDelegatingExecutor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SemaphoredDelegatingExecutor.java
index 4ec77e75ba520..687fa2cf8d6f1 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SemaphoredDelegatingExecutor.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SemaphoredDelegatingExecutor.java
@@ -24,6 +24,8 @@
import com.google.common.util.concurrent.ListeningExecutorService;
import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.statistics.DurationTracker;
+import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
import java.util.Collection;
import java.util.List;
@@ -34,6 +36,10 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
+import static java.util.Objects.requireNonNull;
+import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.stubDurationTrackerFactory;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.ACTION_EXECUTOR_ACQUIRED;
+
/**
* This ExecutorService blocks the submission of new tasks when its queue is
* already full by using a semaphore. Task submissions require permits, task
@@ -54,20 +60,39 @@ public class SemaphoredDelegatingExecutor extends
private final Semaphore queueingPermits;
private final ListeningExecutorService executorDelegatee;
private final int permitCount;
+ private final DurationTrackerFactory trackerFactory;
/**
* Instantiate.
* @param executorDelegatee Executor to delegate to
* @param permitCount number of permits into the queue permitted
* @param fair should the semaphore be "fair"
+ * @param trackerFactory duration tracker factory.
*/
public SemaphoredDelegatingExecutor(
ListeningExecutorService executorDelegatee,
int permitCount,
- boolean fair) {
+ boolean fair,
+ DurationTrackerFactory trackerFactory) {
this.permitCount = permitCount;
queueingPermits = new Semaphore(permitCount, fair);
- this.executorDelegatee = executorDelegatee;
+ this.executorDelegatee = requireNonNull(executorDelegatee);
+ this.trackerFactory = trackerFactory != null
+ ? trackerFactory
+ : stubDurationTrackerFactory();
+ }
+
+ /**
+ * Instantiate without collecting executor aquisition duration information.
+ * @param executorDelegatee Executor to delegate to
+ * @param permitCount number of permits into the queue permitted
+ * @param fair should the semaphore be "fair"
+ */
+ public SemaphoredDelegatingExecutor(
+ ListeningExecutorService executorDelegatee,
+ int permitCount,
+ boolean fair) {
+ this(executorDelegatee, permitCount, fair, null);
}
@Override
@@ -103,7 +128,8 @@ public T invokeAny(Collection extends Callable> tasks, long timeout,
@Override
public ListenableFuture submit(Callable task) {
- try {
+ try (DurationTracker ignored =
+ trackerFactory.trackDuration(ACTION_EXECUTOR_ACQUIRED)) {
queueingPermits.acquire();
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
@@ -114,7 +140,8 @@ public ListenableFuture submit(Callable task) {
@Override
public ListenableFuture submit(Runnable task, T result) {
- try {
+ try (DurationTracker ignored =
+ trackerFactory.trackDuration(ACTION_EXECUTOR_ACQUIRED)) {
queueingPermits.acquire();
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
@@ -125,7 +152,8 @@ public ListenableFuture submit(Runnable task, T result) {
@Override
public ListenableFuture> submit(Runnable task) {
- try {
+ try (DurationTracker ignored =
+ trackerFactory.trackDuration(ACTION_EXECUTOR_ACQUIRED)) {
queueingPermits.acquire();
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
@@ -136,7 +164,8 @@ public ListenableFuture> submit(Runnable task) {
@Override
public void execute(Runnable command) {
- try {
+ try (DurationTracker ignored =
+ trackerFactory.trackDuration(ACTION_EXECUTOR_ACQUIRED)) {
queueingPermits.acquire();
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/BiFunctionRaisingIOE.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/BiFunctionRaisingIOE.java
new file mode 100644
index 0000000000000..ea17c16d01e87
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/BiFunctionRaisingIOE.java
@@ -0,0 +1,40 @@
+/*
+ * 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.util.functional;
+
+import java.io.IOException;
+
+/**
+ * Function of arity 2 which may raise an IOException.
+ * @param type of arg1
+ * @param type of arg2
+ * @param type of return value.
+ */
+@FunctionalInterface
+public interface BiFunctionRaisingIOE {
+
+ /**
+ * Apply the function.
+ * @param t argument 1
+ * @param u argument 2
+ * @return result
+ * @throws IOException Any IO failure
+ */
+ R apply(T t, U u) throws IOException;
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CallableRaisingIOE.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CallableRaisingIOE.java
new file mode 100644
index 0000000000000..65b3a63b2b9a0
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CallableRaisingIOE.java
@@ -0,0 +1,36 @@
+/*
+ * 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.util.functional;
+
+import java.io.IOException;
+
+/**
+ * This is a callable which only raises an IOException.
+ * @param return type
+ */
+@FunctionalInterface
+public interface CallableRaisingIOE {
+
+ /**
+ * Apply the operation.
+ * @return result
+ * @throws IOException Any IO failure
+ */
+ R apply() throws IOException;
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/ConsumerRaisingIOE.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/ConsumerRaisingIOE.java
new file mode 100644
index 0000000000000..24a3b55c58d4a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/ConsumerRaisingIOE.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.util.functional;
+
+import java.io.IOException;
+
+/**
+ * Version of java.util.function.Consumer which raises
+ * exceptions.
+ * @param type of argument,.
+ */
+@FunctionalInterface
+public interface ConsumerRaisingIOE {
+
+ /**
+ * Process the argument.
+ * @param t type
+ * @throws IOException if needed
+ */
+ void accept(T t) throws IOException;
+
+ /**
+ * after calling {@link #accept(Object)},
+ * invoke the next consumer in the chain.
+ * @param next next consumer
+ * @return the chain.
+ */
+ default ConsumerRaisingIOE andThen(
+ ConsumerRaisingIOE super T> next) {
+ return (T t) -> {
+ accept(t);
+ next.accept(t);
+ };
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FunctionRaisingIOE.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FunctionRaisingIOE.java
new file mode 100644
index 0000000000000..51a7839be4995
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FunctionRaisingIOE.java
@@ -0,0 +1,39 @@
+/*
+ * 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.util.functional;
+
+import java.io.IOException;
+
+/**
+ * Function of arity 1 which may raise an IOException.
+ * @param type of arg1
+ * @param type of return value.
+ */
+@FunctionalInterface
+public interface FunctionRaisingIOE {
+
+ /**
+ * Apply the function.
+ * @param t argument 1
+ * @return result
+ * @throws IOException Any IO failure
+ */
+ R apply(T t) throws IOException;
+}
+
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RemoteIterators.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RemoteIterators.java
new file mode 100644
index 0000000000000..47a04f101256c
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RemoteIterators.java
@@ -0,0 +1,594 @@
+/*
+ * 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.util.functional;
+
+import javax.annotation.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Objects;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+
+import static java.util.Objects.requireNonNull;
+import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.logIOStatisticsAtDebug;
+import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics;
+
+/**
+ * A set of remote iterators supporting transformation and filtering,
+ * with IOStatisticsSource passthrough, and of conversions of
+ * the iterators to lists/arrays and of performing actions
+ * on the values.
+ *
+ * This aims to make it straightforward to use lambda-expressions to
+ * transform the results of an iterator, without losing the statistics
+ * in the process, and to chain the operations together.
+ *
+ * The closeable operation will be passed through RemoteIterators which
+ * wrap other RemoteIterators. This is to support any iterator which
+ * can be closed to release held connections, file handles etc.
+ * Unless client code is written to assume that RemoteIterator instances
+ * may be closed, this is not likely to be broadly used. It is added
+ * to make it possible to adopt this feature in a managed way.
+ *
+ * One notable feature is that the
+ * {@link #foreach(RemoteIterator, ConsumerRaisingIOE)} method will
+ * LOG at debug any IOStatistics provided by the iterator, if such
+ * statistics are provided. There's no attempt at retrieval and logging
+ * if the LOG is not set to debug, so it is a zero cost feature unless
+ * the logger {@code org.apache.hadoop.fs.functional.RemoteIterators}
+ * is at DEBUG.
+ *
+ * Based on the S3A Listing code, and some some work on moving other code
+ * to using iterative listings so as to pick up the statistics.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public final class RemoteIterators {
+
+ /**
+ * Log used for logging any statistics in
+ * {@link #foreach(RemoteIterator, ConsumerRaisingIOE)}
+ * at DEBUG.
+ */
+ private static final Logger LOG = LoggerFactory.getLogger(
+ RemoteIterators.class);
+
+ private RemoteIterators() {
+ }
+
+ /**
+ * Create an iterator from a singleton.
+ * @param singleton instance
+ * @param type
+ * @return a remote iterator
+ */
+ public static RemoteIterator remoteIteratorFromSingleton(
+ @Nullable T singleton) {
+ return new SingletonIterator<>(singleton);
+ }
+
+ /**
+ * Create a remote iterator from a java.util.Iterator.
+ * @param type
+ * @return a remote iterator
+ */
+ public static RemoteIterator remoteIteratorFromIterator(
+ Iterator iterator) {
+ return new FromIterator<>(iterator);
+ }
+
+ /**
+ * Create a remote iterator from a java.util.Iterable -e.g. a list
+ * or other collection.
+ * @param type
+ * @return a remote iterator
+ */
+ public static RemoteIterator remoteIteratorFromIterable(
+ Iterable iterable) {
+ return new FromIterator<>(iterable.iterator());
+ }
+
+ /**
+ * Create a remote iterator from an array.
+ * @param type
+ * @return a remote iterator
+ */
+ public static RemoteIterator remoteIteratorFromArray(T[] array) {
+ return new FromIterator<>(Arrays.stream(array).iterator());
+ }
+
+ /**
+ * Create an iterator from an iterator and a transformation function.
+ * @param source type
+ * @param result type
+ * @param iterator source
+ * @param mapper transformation
+ * @return a remote iterator
+ */
+ public static RemoteIterator mappingRemoteIterator(
+ RemoteIterator iterator,
+ FunctionRaisingIOE super S, T> mapper) {
+ return new MappingRemoteIterator<>(iterator, mapper);
+ }
+
+ /**
+ * Create an iterator from an iterator and a filter.
+ *
+ * Elements are filtered in the hasNext() method; if not used
+ * the filtering will be done on demand in the {@code next()}
+ * call.
+ * @param type
+ * @param iterator source
+ * @param filter filter
+ * @return a remote iterator
+ */
+ public static RemoteIterator filteringRemoteIterator(
+ RemoteIterator iterator,
+ FunctionRaisingIOE super S, Boolean> filter) {
+ return new FilteringRemoteIterator<>(iterator, filter);
+ }
+
+ /**
+ * This adds an extra close operation alongside the passthrough
+ * to any Closeable.close() method supported by the source iterator.
+ * @param iterator source
+ * @param toClose extra object to close.
+ * @param source type.
+ * @return a new iterator
+ */
+ public static RemoteIterator closingRemoteIterator(
+ RemoteIterator iterator,
+ Closeable toClose) {
+ return new CloseRemoteIterator<>(iterator, toClose);
+ }
+
+ /**
+ * Build a list from a RemoteIterator.
+ * @param type
+ * @return a list of the values.
+ * @throws IOException if the source RemoteIterator raises it.
+ */
+ public static List toList(RemoteIterator source)
+ throws IOException {
+ List l = new ArrayList<>();
+ foreach(source, l::add);
+ return l;
+ }
+
+ /**
+ * Build an array from a RemoteIterator.
+ * @param type
+ * @return an array of the values.
+ * @throws IOException if the source RemoteIterator raises it.
+ */
+ public static T[] toArray(RemoteIterator source) throws IOException {
+ return (T[]) toList(source).toArray();
+ }
+
+ /**
+ * Apply an operation to all values of a RemoteIterator.
+ *
+ * If the iterator is an IOStatisticsSource returning a non-null
+ * set of statistics, and this classes log is set to DEBUG,
+ * then the statistics of the operation are evaluated and logged at
+ * debug.
+ *
+ * The number of entries processed is returned, as it is useful to
+ * know this, especially during tests or when reporting values
+ * to users.
+ *
+ * This does not close the iterator afterwards.
+ * @param source iterator source
+ * @param consumer consumer of the values.
+ * @return the number of elements processed
+ * @param type of source
+ * @throws IOException if the source RemoteIterator or the consumer raise one.
+ */
+ public static long foreach(
+ RemoteIterator source,
+ ConsumerRaisingIOE super T> consumer) throws IOException {
+ long count = 0;
+
+ while (source.hasNext()) {
+ count++;
+ consumer.accept(source.next());
+ }
+
+ // maybe log the results
+ logIOStatisticsAtDebug(LOG, "RemoteIterator Statistics: {}", source);
+ return count;
+ }
+
+
+ /**
+ * A remote iterator from a singleton. It has a single next()
+ * value, after which hasNext() returns false and next() fails.
+ *
+ * If it is a source of
+ * remote statistics, these are returned.
+ * @param type.
+ */
+ private static final class SingletonIterator
+ implements RemoteIterator, IOStatisticsSource {
+
+ /**
+ * Single entry.
+ */
+ private T singleton;
+
+ /** Has the entry been processed? */
+ private boolean processed;
+
+ /**
+ * Instantiate.
+ * @param singleton single value...may be null
+ */
+ private SingletonIterator(@Nullable T singleton) {
+ this.singleton = singleton;
+ // if the entry is null, consider it processed.
+ this.processed = singleton == null;
+ }
+
+ @Override
+ public boolean hasNext() throws IOException {
+ return !processed;
+ }
+
+ @SuppressWarnings("NewExceptionWithoutArguments")
+ @Override
+ public T next() throws IOException {
+ if (hasNext()) {
+ processed = true;
+ return singleton;
+ } else {
+ throw new NoSuchElementException();
+ }
+ }
+
+ @Override
+ public IOStatistics getIOStatistics() {
+ return retrieveIOStatistics(singleton);
+ }
+
+ @Override
+ public String toString() {
+ return "SingletonIterator{"
+ + (singleton != null ? singleton : "")
+ + '}';
+ }
+
+ }
+
+ /**
+ * Create a remote iterator from a simple java.util.Iterator, or
+ * an iterable.
+ *
+ * If the iterator is a source of statistics that is passed through.
+ *
+ * The {@link #close()} will close the source iterator if it is
+ * Closeable;
+ * @param iterator type.
+ */
+ private static final class FromIterator
+ implements RemoteIterator, IOStatisticsSource, Closeable {
+
+ /**
+ * inner iterator..
+ */
+ private final Iterator extends T> source;
+
+ private final Closeable sourceToClose;
+
+
+ /**
+ * Construct from an interator.
+ * @param source source iterator.
+ */
+ private FromIterator(Iterator extends T> source) {
+ this.source = requireNonNull(source);
+ sourceToClose = new MaybeClose(source);
+ }
+
+ @Override
+ public boolean hasNext() {
+ return source.hasNext();
+ }
+
+ @Override
+ public T next() {
+ return source.next();
+ }
+
+ @Override
+ public IOStatistics getIOStatistics() {
+ return retrieveIOStatistics(source);
+ }
+
+ @Override
+ public String toString() {
+ return "FromIterator{" + source + '}';
+ }
+
+ @Override
+ public void close() throws IOException {
+ sourceToClose.close();
+
+ }
+ }
+
+ /**
+ * Wrapper of another remote iterator; IOStatistics
+ * and Closeable methods are passed down if implemented.
+ * @param source type
+ * @param type of returned value
+ */
+ private static abstract class WrappingRemoteIterator
+ implements RemoteIterator, IOStatisticsSource, Closeable {
+
+ /**
+ * Source iterator.
+ */
+ private final RemoteIterator source;
+
+ private final Closeable sourceToClose;
+
+ protected WrappingRemoteIterator(final RemoteIterator source) {
+ this.source = requireNonNull(source);
+ sourceToClose = new MaybeClose(source);
+ }
+
+ protected RemoteIterator getSource() {
+ return source;
+ }
+
+ @Override
+ public IOStatistics getIOStatistics() {
+ return retrieveIOStatistics(source);
+ }
+
+ @Override
+ public void close() throws IOException {
+ sourceToClose.close();
+ }
+
+ @Override
+ public String toString() {
+ return source.toString();
+ }
+
+ }
+
+ /**
+ * Iterator taking a source and a transformational function.
+ * @param source type
+ * @param final output type.There
+ */
+ private static final class MappingRemoteIterator
+ extends WrappingRemoteIterator {
+
+ /**
+ * Mapper to invoke.
+ */
+ private final FunctionRaisingIOE super S, T> mapper;
+
+ private MappingRemoteIterator(
+ RemoteIterator source,
+ FunctionRaisingIOE super S, T> mapper) {
+ super(source);
+ this.mapper = requireNonNull(mapper);
+ }
+
+ @Override
+ public boolean hasNext() throws IOException {
+ return getSource().hasNext();
+ }
+
+ @Override
+ public T next() throws IOException {
+ return mapper.apply(getSource().next());
+ }
+
+ @Override
+ public String toString() {
+ return "FunctionRemoteIterator{" + getSource() + '}';
+ }
+ }
+
+ /**
+ * Extend the wrapped iterator by filtering source values out.
+ * Only those values for which the filter predicate returns true
+ * will be returned.
+ * @param type of iterator.
+ */
+ @SuppressWarnings("NewExceptionWithoutArguments")
+ private static final class FilteringRemoteIterator
+ extends WrappingRemoteIterator {
+
+ /**
+ * Filter Predicate.
+ * Takes the input type or any superclass.
+ */
+ private final FunctionRaisingIOE super S, Boolean>
+ filter;
+
+ /**
+ * Next value; will be null if none has been evaluated, or the
+ * last one was already returned by next().
+ */
+ private S next;
+
+ /**
+ * An iterator which combines filtering with transformation.
+ * All source elements for which filter = true are returned,
+ * transformed via the mapper.
+ * @param source source iterator.
+ * @param filter filter predicate.
+ */
+ private FilteringRemoteIterator(
+ RemoteIterator source,
+ FunctionRaisingIOE super S, Boolean> filter) {
+ super(source);
+
+ this.filter = requireNonNull(filter);
+ }
+
+ /**
+ * Fetch: retrieve the next value.
+ * @return true if a new value was found after filtering.
+ * @throws IOException failure in retrieval from source or mapping
+ */
+ private boolean fetch() throws IOException {
+ while (next == null && getSource().hasNext()) {
+ S candidate = getSource().next();
+ if (filter.apply(candidate)) {
+ next = candidate;
+ return true;
+ }
+ }
+ return false;
+ }
+
+ /**
+ * Trigger a fetch if an entry is needed.
+ * @return true if there was already an entry return,
+ * or there was not but one could then be retrieved.set
+ * @throws IOException failure in fetch operation
+ */
+ @Override
+ public boolean hasNext() throws IOException {
+ if (next != null) {
+ return true;
+ }
+ return fetch();
+ }
+
+ /**
+ * Return the next value.
+ * Will retrieve the next elements if needed.
+ * This is where the mapper takes place.
+ * @return true if there is another data element.
+ * @throws IOException failure in fetch operation or the transformation.
+ * @throws NoSuchElementException no more data
+ */
+ @Override
+ public S next() throws IOException {
+ if (hasNext()) {
+ S result = next;
+ next = null;
+ return result;
+ }
+ throw new NoSuchElementException();
+ }
+
+ @Override
+ public String toString() {
+ return "FilteringRemoteIterator{" + getSource() + '}';
+ }
+ }
+
+ /**
+ * A wrapping remote iterator which adds another entry to
+ * close. This is to assist cleanup.
+ * @param type
+ */
+ private static final class CloseRemoteIterator
+ extends WrappingRemoteIterator {
+
+ private final MaybeClose toClose;
+
+ private CloseRemoteIterator(
+ final RemoteIterator source,
+ final Closeable toClose) {
+ super(source);
+ this.toClose = new MaybeClose(Objects.requireNonNull(toClose));
+ }
+
+ @Override
+ public boolean hasNext() throws IOException {
+ return getSource().hasNext();
+ }
+
+ @Override
+ public S next() throws IOException {
+ return getSource().next();
+ }
+
+ @Override
+ public void close() throws IOException {
+ try {
+ super.close();
+ } finally {
+ toClose.close();
+ }
+ }
+ }
+
+ /**
+ * Class to help with Closeable logic, where sources may/may not
+ * be closeable, only one invocation is allowed.
+ */
+ private static final class MaybeClose implements Closeable {
+
+ private Closeable toClose;
+
+ /**
+ * Construct.
+ * @param o object to close.
+ */
+ private MaybeClose(Object o) {
+ this(o, true);
+ }
+
+ /**
+ * Construct -close the object if it is closeable and close==true
+ * @param o object to close.
+ * @param close should close?
+ */
+ private MaybeClose(Object o, boolean close) {
+ if (close && o instanceof Closeable) {
+ this.toClose = (Closeable) o;
+ } else {
+ this.toClose = null;
+ }
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (toClose != null) {
+ try {
+ toClose.close();
+ } finally {
+ toClose = null;
+ }
+ }
+ }
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FunctionsRaisingIOE.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RuntimeIOException.java
similarity index 51%
rename from hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FunctionsRaisingIOE.java
rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RuntimeIOException.java
index 7bbb34622647d..ae93d0c48fd74 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FunctionsRaisingIOE.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RuntimeIOException.java
@@ -16,54 +16,41 @@
* limitations under the License.
*/
-package org.apache.hadoop.fs.impl;
+package org.apache.hadoop.util.functional;
import java.io.IOException;
+import java.util.Objects;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
- * Evolving support for functional programming/lambda-expressions.
+ * A RuntimeException which always contains an IOException.
+ *
+ * The constructor signature guarantees the cause will be an IOException,
+ * and as it checks for a null-argument, non-null.
*/
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public final class FunctionsRaisingIOE {
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class RuntimeIOException extends RuntimeException {
- private FunctionsRaisingIOE() {
- }
+ private static final long serialVersionUID = 7543456147436005830L;
/**
- * Function of arity 1 which may raise an IOException.
- * @param type of arg1
- * @param type of return value.
+ * Construct from a non-null IOException.
+ * @param cause inner cause
+ * @throws NullPointerException if the cause is null.
*/
- @FunctionalInterface
- public interface FunctionRaisingIOE {
-
- R apply(T t) throws IOException;
+ public RuntimeIOException(final IOException cause) {
+ super(Objects.requireNonNull(cause));
}
/**
- * Function of arity 2 which may raise an IOException.
- * @param type of arg1
- * @param type of arg2
- * @param type of return value.
+ * Return the cause, cast to an IOException.
+ * @return cause of this exception.
*/
- @FunctionalInterface
- public interface BiFunctionRaisingIOE {
-
- R apply(T t, U u) throws IOException;
+ @Override
+ public synchronized IOException getCause() {
+ return (IOException) super.getCause();
}
-
- /**
- * This is a callable which only raises an IOException.
- * @param return type
- */
- @FunctionalInterface
- public interface CallableRaisingIOE {
-
- R apply() throws IOException;
- }
-
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/package-info.java
new file mode 100644
index 0000000000000..1c204bb9979a8
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/package-info.java
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+/**
+ * Support for functional programming within the Hadoop APIs.
+ *
+ * Much of this is needed simply to cope with Java's checked exceptions and
+ * the fact that the java.util.function can only throw runtime exceptions.
+ *
+ * Pretty much all the Hadoop FS APIs raise IOExceptions, hence the need
+ * for these classes. If Java had made a different decision about the
+ * nature of exceptions, life would be better.
+ *
+ * Do note that the {@link org.apache.hadoop.util.functional.RemoteIterators}
+ * iterators go beyond that of the java ones, in terms of declaring themselves
+ * Closeable and implementors of
+ * {@link org.apache.hadoop.fs.statistics.IOStatisticsSource}; a chain
+ * of wrapped iterators can supply statistics of the inner iterators, and
+ * encourage close() to be called after use.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+package org.apache.hadoop.util.functional;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md
index df538ee6cf96b..25cc9d13d1fca 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md
@@ -38,3 +38,4 @@ HDFS as these are commonly expected by Hadoop client applications.
2. [Testing with the Filesystem specification](testing.html)
2. [Extending the specification and its tests](extending.html)
1. [Uploading a file using Multiple Parts](multipartuploader.html)
+1. [IOStatistics](iostatistics.html)
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/iostatistics.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/iostatistics.md
new file mode 100644
index 0000000000000..c8de122cd8aed
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/iostatistics.md
@@ -0,0 +1,432 @@
+
+
+# Statistic collection with the IOStatistics API
+
+```java
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+```
+
+The `IOStatistics` API is intended to provide statistics on individual IO
+classes -such as input and output streams, *in a standard way which
+applications can query*
+
+Many filesystem-related classes have implemented statistics gathering
+and provided private/unstable ways to query this, but as they were
+not common across implementations it was unsafe for applications
+to reference these values. Example: `S3AInputStream` and its statistics
+API. This is used in internal tests, but cannot be used downstream in
+applications such as Apache Hive or Apache HBase.
+
+The IOStatistics API is intended to
+
+1. Be instance specific:, rather than shared across multiple instances
+ of a class, or thread local.
+1. Be public and stable enough to be used by applications.
+1. Be easy to use in applications written in Java, Scala, and, via libhdfs, C/C++
+1. Have foundational interfaces and classes in the `hadoop-common` JAR.
+
+## Core Model
+
+Any class *may* implement `IOStatisticsSource` in order to
+provide statistics.
+
+Wrapper I/O Classes such as `FSDataInputStream` anc `FSDataOutputStream` *should*
+implement the interface and forward it to the wrapped class, if they also
+implement it -and return `null` if they do not.
+
+`IOStatisticsSource` implementations `getIOStatistics()` return an
+instance of `IOStatistics` enumerating the statistics of that specific
+instance.
+
+The `IOStatistics` Interface exports five kinds of statistic:
+
+
+| Category | Type | Description |
+|------|------|-------------|
+| `counter` | `long` | a counter which may increase in value; SHOULD BE >= 0 |
+| `gauge` | `long` | an arbitrary value which can down as well as up; SHOULD BE >= 0|
+| `minimum` | `long` | an minimum value; MAY BE negative |
+| `maximum` | `long` | a maximum value; MAY BE negative |
+| `meanStatistic` | `MeanStatistic` | an arithmetic mean and sample size; mean MAY BE negative|
+
+Four are simple `long` values, with the variations how they are likely to
+change and how they are aggregated.
+
+
+#### Aggregation of Statistic Values
+
+For the different statistic category, the result of `aggregate(x, y)` is
+
+| Category | Aggregation |
+|------------------|-------------|
+| `counter` | `max(0, x) + max(0, y)` |
+| `gauge` | `max(0, x) + max(0, y)` |
+| `minimum` | `min(x, y)` |
+| `maximum` | `max(x, y)` |
+| `meanStatistic` | calculation of the mean of `x` and `y` ) |
+
+
+#### Class `MeanStatistic`
+
+## package `org.apache.hadoop.fs.statistics`
+
+This package contains the public statistics APIs intended
+for use by applications.
+
+
+
+
+
+`MeanStatistic` is a tuple of `(mean, samples)` to support aggregation.
+
+A `MeanStatistic` with a sample of `0` is considered an empty statistic.
+
+All `MeanStatistic` instances where `sample = 0` are considered equal,
+irrespective of the `mean` value.
+
+Algorithm to calculate the mean :
+
+```python
+if x.samples = 0:
+ y
+else if y.samples = 0 :
+ x
+else:
+ samples' = x.samples + y.samples
+ mean' = (x.mean * x.samples) + (y.mean * y.samples) / samples'
+ (samples', mean')
+```
+
+Implicitly, this means that if both samples are empty, then the aggregate value is also empty.
+
+```java
+public final class MeanStatistic implements Serializable, Cloneable {
+ /**
+ * Arithmetic mean.
+ */
+ private double mean;
+
+ /**
+ * Number of samples used to calculate
+ * the mean.
+ */
+ private long samples;
+
+ /**
+ * Get the mean value.
+ * @return the mean
+ */
+ public double getMean() {
+ return mean;
+ }
+
+ /**
+ * Get the sample count.
+ * @return the sample count; 0 means empty
+ */
+ public long getSamples() {
+ return samples;
+ }
+
+ /**
+ * Is a statistic empty?
+ * @return true if the sample count is 0
+ */
+ public boolean isEmpty() {
+ return samples == 0;
+ }
+ /**
+ * Add another mean statistic to create a new statistic.
+ * When adding two statistics, if either is empty then
+ * a copy of the non-empty statistic is returned.
+ * If both are empty then a new empty statistic is returned.
+ *
+ * @param other other value
+ * @return the aggregate mean
+ */
+ public MeanStatistic add(final MeanStatistic other) {
+ /* Implementation elided. */
+ }
+ @Override
+ public int hashCode() {
+ return Objects.hash(mean, samples);
+ }
+
+ @Override
+ public boolean equals(final Object o) {
+ if (this == o) { return true; }
+ if (o == null || getClass() != o.getClass()) { return false; }
+ MeanStatistic that = (MeanStatistic) o;
+ if (this.isEmpty()) {
+ return that.isEmpty();
+ }
+ return Double.compare(that.mean, mean) == 0 &&
+ samples == that.samples;
+ }
+
+ @Override
+ public MeanStatistic clone() {
+ return new MeanStatistic(this);
+ }
+
+ public MeanStatistic copy() {
+ return new MeanStatistic(this);
+ }
+
+}
+```
+
+
+
+
+
+### class `org.apache.hadoop.fs.statistics.IOStatisticsSource`
+
+```java
+
+/**
+ * A source of IO statistics.
+ * These statistics MUST be instance specific, not thread local.
+ */
+@InterfaceStability.Unstable
+public interface IOStatisticsSource {
+
+ /**
+ * Return a statistics instance.
+ * It is not a requirement that the same instance is returned every time.
+ * {@link IOStatisticsSource}.
+ * If the object implementing this is Closeable, this method
+ * may return null if invoked on a closed object, even if
+ * it returns a valid instance when called earlier.
+ * @return an IOStatistics instance or null
+ */
+ IOStatistics getIOStatistics();
+}
+```
+
+This is the interface which an object instance MUST implement if they are a source of
+IOStatistics information.
+
+#### Invariants
+
+The result of `getIOStatistics()` must be one of
+
+* `null`
+* an immutable `IOStatistics` for which each map of entries is
+an empty map.
+* an instance of an `IOStatistics` whose statistics MUST BE unique to that
+instance of the class implementing `IOStatisticsSource`.
+
+Less formally: if the statistics maps returned are non-empty, all the statistics
+must be collected from the current instance, and not from any other instances, the way
+some of the `FileSystem` statistics are collected.
+
+
+The result of `getIOStatistics()`, if non-null, MAY be a different instance
+on every invocation.
+
+
+
+
+
+
+### class `org.apache.hadoop.fs.statistics.IOStatistics`
+
+These are per-instance statistics provided by an object which
+implements `IOStatisticsSource`.
+
+```java
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public interface IOStatistics {
+
+ /**
+ * Map of counters.
+ * @return the current map of counters.
+ */
+ Map counters();
+
+ /**
+ * Map of gauges.
+ * @return the current map of gauges.
+ */
+ Map gauges();
+
+ /**
+ * Map of minumums.
+ * @return the current map of minumums.
+ */
+ Map minumums();
+
+ /**
+ * Map of maximums.
+ * @return the current map of maximums.
+ */
+ Map maximums();
+
+ /**
+ * Map of meanStatistics.
+ * @return the current map of MeanStatistic statistics.
+ */
+ Map meanStatistics();
+
+}
+```
+
+### Statistic Naming
+
+The naming policy of statistics is designed to be readable, shareable
+and ideally consistent across `IOStatisticSource` implementations.
+
+* Characters in key names MUST match the regular expression
+ `[a-z|0-9|_]` with the exception of the first character, which
+ MUST be in the range `[a-z]`. Thus the full regular expression
+ for a valid statistic name is:
+
+ [a-z][a-z|0-9|_]+
+
+* Where possible, the names of statistics SHOULD be those defined
+ with common names.
+
+ org.apache.hadoop.fs.statistics.StreamStatisticNames
+ org.apache.hadoop.fs.statistics.StoreStatisticNames
+
+ Note 1.: these are evolving; for clients to safely reference their
+ statistics by name they SHOULD be copied to the application.
+ (i.e. for an application compiled hadoop 3.4.2 to link against hadoop 3.4.1,
+ copy the strings).
+
+ Note 2: keys defined in these classes SHALL NOT be removed
+ from subsequent Hadoop releases.
+
+* A common statistic name MUST NOT be used to report any other statistic and
+ MUST use the pre-defined unit of measurement.
+
+* A statistic name in one of the maps SHOULD NOT be re-used in another map.
+ This aids diagnostics of logged statistics.
+
+### Statistic Maps
+
+For each map of statistics returned:
+
+* The operations to add/remove entries are unsupported: the map returned
+ MAY be mutable by the source of statistics.
+
+* The map MAY be empty.
+
+* The map keys each represent a measured statistic.
+
+* The set of keys in a map SHOULD remain unchanged, and MUST NOT remove keys.
+
+* The statistics SHOULD be dynamic: every lookup of an entry SHOULD
+ return the latest value.
+
+* The values MAY change across invocations of `Map.values()` and `Map.entries()`
+
+* The update MAY be in the `iterable()` calls of the iterators returned,
+ or MAY be in the actual `iterable.next()` operation. That is: there is
+ no guarantee as to when the evaluation takes place.
+
+* The returned `Map.Entry` instances MUST return the same value on
+ repeated `getValue()` calls. (i.e once you have the entry, it is immutable).
+
+* Queries of statistics SHOULD be fast and non-blocking to the extent
+ that if invoked during a long operation, they will prioritize
+ returning fast over most timely values.
+
+* The statistics MAY lag; especially for statistics collected in separate
+ operations (e.g stream IO statistics as provided by a filesystem
+ instance).
+
+* Statistics which represent time SHOULD use milliseconds as their unit.
+
+* Statistics which represent time and use a different unit MUST document
+ the unit used.
+
+### Thread Model
+
+1. An instance of `IOStatistics` can be shared across threads;
+
+1. Read access to the supplied statistics maps MUST be thread safe.
+
+1. Iterators returned from the maps MUST NOT be shared across threads.
+
+1. The statistics collected MUST include all operations which took
+ place across all threads performing work for the monitored object.
+
+1. The statistics reported MUST NOT be local to the active thread.
+
+This is different from the `FileSystem.Statistics` behavior where per-thread statistics
+are collected and reported.
+
+That mechanism supports collecting limited read/write statistics for different
+worker threads sharing the same FS instance, but as the collection is thread local,
+it invariably under-reports IO performed in other threads on behalf of a worker thread.
+
+
+## Statisic Snapshot
+
+A snapshot of the current statistic values MAY be obtained by calling
+`IOStatisticsSupport.snapshotIOStatistics()`
+
+```java
+ public static X
+ snapshotIOStatistics(IOStatistics statistics)
+```
+
+This snapshot is serializable through Java serialization and through
+Jackson to/from JSON.
+
+## Helper Classes
+
+
+### class `org.apache.hadoop.fs.statistics.IOStatisticsSupport`
+
+This provides helper methods to work with IOStatistics sources and instances.
+
+Consult the javadocs for its operations.
+
+### class `org.apache.hadoop.fs.statistics.IOStatisticsLogging`
+
+Support for efficiently logging `IOStatistics`/`IOStatisticsSource`
+instances.
+
+These are intended for assisting logging, including only enumerating the
+state of an `IOStatistics` instance when the log level needs it.
+
+```java
+LOG.info("IOStatistics after upload: {}", demandStringify(iostats));
+
+// or even better, as it results in only a single object creations
+Object latest = demandStringify(iostats);
+LOG.info("IOStatistics : {}", latest);
+/* do some work. */
+LOG.info("IOStatistics : {}", latest);
+
+```
+
+## Package `org.apache.hadoop.fs.statistics.impl`
+
+This contains implementation classes to support providing statistics to applications.
+
+These MUST NOT BE used by applications. If a feature is needed from this package then
+the provisioning of a public implementation MAY BE raised via the Hadoop development
+channels.
+
+These MAY be used by those implementations of the Hadoop `FileSystem`, `AbstractFileSystem`
+and related classes which are not in the hadoop source tree. Implementors need to
+be aware that the implementation this code is unstable and may change across
+minor point releases of Hadoop.
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java
index 8a520fae8a2c3..f08fab3972964 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java
@@ -51,6 +51,7 @@
import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyPathExists;
import static org.apache.hadoop.fs.impl.FutureIOSupport.awaitFuture;
+import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString;
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
import static org.apache.hadoop.test.LambdaTestUtils.eventually;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
@@ -106,6 +107,8 @@ public void teardown() throws Exception {
CompletableFuture f
= uploader.abortUploadsUnderPath(teardown);
f.get();
+ LOG.info("Statistics {}",
+ ioStatisticsSourceToString(uploader));
} catch (Exception e) {
LOG.warn("Exeception in teardown", e);
}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractStreamIOStatisticsTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractStreamIOStatisticsTest.java
new file mode 100644
index 0000000000000..24d0c2840caa0
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractStreamIOStatisticsTest.java
@@ -0,0 +1,282 @@
+/*
+ * 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.contract;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsLogging;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.extractStatistics;
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyCounterStatisticValue;
+import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatisticsSource;
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BYTES;
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_WRITE_BYTES;
+
+/**
+ * Tests {@link IOStatistics} support in input and output streams.
+ *
+ * Requires both the input and output streams to offer the basic
+ * bytes read/written statistics.
+ *
+ * If the IO is buffered, that information must be provided,
+ * especially the input buffer size.
+ */
+public abstract class AbstractContractStreamIOStatisticsTest
+ extends AbstractFSContractTestBase {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(AbstractContractStreamIOStatisticsTest.class);
+
+ @Test
+ public void testOutputStreamStatisticKeys() throws Throwable {
+ describe("Look at the statistic keys of an output stream");
+ Path path = methodPath();
+ FileSystem fs = getFileSystem();
+ fs.mkdirs(path.getParent());
+ try (FSDataOutputStream out = fs.create(path, true)) {
+ IOStatistics statistics = extractStatistics(out);
+ final List keys = outputStreamStatisticKeys();
+ Assertions.assertThat(statistics.counters().keySet())
+ .describedAs("statistic keys of %s", statistics)
+ .containsAll(keys);
+ Assertions.assertThat(keys)
+ .describedAs("Statistics supported by the stream %s", out)
+ .contains(STREAM_WRITE_BYTES);
+ } finally {
+ fs.delete(path, false);
+ }
+ }
+
+ /**
+ * If the stream writes in blocks, then counters during the write may be
+ * zero until a whole block is written -or the write has finished.
+ * @return true if writes are buffered into whole blocks.
+ */
+ public boolean streamWritesInBlocks() {
+ return false;
+ }
+
+ @Test
+ public void testWriteSingleByte() throws Throwable {
+ describe("Write a byte to a file and verify"
+ + " the stream statistics are updated");
+ Path path = methodPath();
+ FileSystem fs = getFileSystem();
+ fs.mkdirs(path.getParent());
+ boolean writesInBlocks = streamWritesInBlocks();
+ try (FSDataOutputStream out = fs.create(path, true)) {
+ IOStatistics statistics = extractStatistics(out);
+ // before a write, no bytes
+ verifyCounterStatisticValue(statistics, STREAM_WRITE_BYTES, 0);
+ out.write('0');
+ verifyCounterStatisticValue(statistics, STREAM_WRITE_BYTES,
+ writesInBlocks ? 0 : 1);
+ // close the stream
+ out.close();
+ // statistics are still valid after the close
+ // always call the output stream to check that behavior
+ statistics = extractStatistics(out);
+ final String strVal = statistics.toString();
+ LOG.info("Statistics = {}", strVal);
+ verifyCounterStatisticValue(statistics, STREAM_WRITE_BYTES, 1);
+ } finally {
+ fs.delete(path, false);
+ }
+ }
+
+ @Test
+ public void testWriteByteArrays() throws Throwable {
+ describe("Write byte arrays to a file and verify"
+ + " the stream statistics are updated");
+ Path path = methodPath();
+ FileSystem fs = getFileSystem();
+ fs.mkdirs(path.getParent());
+ boolean writesInBlocks = streamWritesInBlocks();
+ try (FSDataOutputStream out = fs.create(path, true)) {
+ Object demandStatsString = demandStringifyIOStatisticsSource(out);
+ // before a write, no bytes
+ final byte[] bytes = ContractTestUtils.toAsciiByteArray(
+ "statistically-speaking");
+ final long len = bytes.length;
+ out.write(bytes);
+ out.flush();
+ LOG.info("stats {}", demandStatsString);
+ IOStatistics statistics = extractStatistics(out);
+ verifyCounterStatisticValue(statistics, STREAM_WRITE_BYTES,
+ writesInBlocks ? 0 : len);
+ out.write(bytes);
+ out.flush();
+ verifyCounterStatisticValue(statistics, STREAM_WRITE_BYTES,
+ writesInBlocks ? 0 : len * 2);
+ // close the stream
+ out.close();
+ LOG.info("stats {}", demandStatsString);
+ // statistics are still valid after the close
+ // always call the output stream to check that behavior
+ statistics = extractStatistics(out);
+ verifyCounterStatisticValue(statistics, STREAM_WRITE_BYTES, len * 2);
+ // the to string value must contain the same counterHiCable you mean
+ Assertions.assertThat(demandStatsString.toString())
+ .contains(Long.toString(len * 2));
+ } finally {
+ fs.delete(path, false);
+ }
+ }
+
+ @Test
+ public void testInputStreamStatisticKeys() throws Throwable {
+ describe("Look at the statistic keys of an input stream");
+ Path path = methodPath();
+ FileSystem fs = getFileSystem();
+ ContractTestUtils.touch(fs, path);
+ try (FSDataInputStream in = fs.open(path)) {
+ IOStatistics statistics = extractStatistics(in);
+ final List keys = inputStreamStatisticKeys();
+ Assertions.assertThat(statistics.counters().keySet())
+ .describedAs("statistic keys of %s", statistics)
+ .containsAll(keys);
+ Assertions.assertThat(keys)
+ .describedAs("Statistics supported by the stream %s", in)
+ .contains(STREAM_READ_BYTES);
+ verifyCounterStatisticValue(statistics, STREAM_READ_BYTES, 0);
+ } finally {
+ fs.delete(path, false);
+ }
+ }
+
+ @Test
+ public void testInputStreamStatisticRead() throws Throwable {
+ describe("Read Data from an input stream");
+ Path path = methodPath();
+ FileSystem fs = getFileSystem();
+ final int fileLen = 1024;
+ final byte[] ds = dataset(fileLen, 'a', 26);
+ ContractTestUtils.writeDataset(fs, path, ds, fileLen, 8_000, true);
+
+ try (FSDataInputStream in = fs.open(path)) {
+ long current = 0;
+ IOStatistics statistics = extractStatistics(in);
+ verifyCounterStatisticValue(statistics, STREAM_READ_BYTES, 0);
+ Assertions.assertThat(in.read()).isEqualTo('a');
+ int bufferSize = readBufferSize();
+ // either a single byte was read or a whole block
+ current = verifyBytesRead(statistics, current, 1, bufferSize);
+ final int bufferLen = 128;
+ byte[] buf128 = new byte[bufferLen];
+ in.read(buf128);
+ current = verifyBytesRead(statistics, current, bufferLen, bufferSize);
+ in.readFully(buf128);
+ current = verifyBytesRead(statistics, current, bufferLen, bufferSize);
+ in.readFully(0, buf128);
+ current = verifyBytesRead(statistics, current, bufferLen, bufferSize);
+ // seek must not increment the read counter
+ in.seek(256);
+ verifyBytesRead(statistics, current, 0, bufferSize);
+
+ // if a stream implements lazy-seek the seek operation
+ // may be postponed until the read
+ final int sublen = 32;
+ Assertions.assertThat(in.read(buf128, 0, sublen))
+ .isEqualTo(sublen);
+ current = verifyBytesRead(statistics, current, sublen, bufferSize);
+
+ // perform some read operations near the end of the file such that
+ // the buffer will not be completely read.
+ // skip these tests for buffered IO as it is too complex to work out
+ if (bufferSize == 0) {
+ final int pos = fileLen - sublen;
+ in.seek(pos);
+ Assertions.assertThat(in.read(buf128))
+ .describedAs("Read overlapping EOF")
+ .isEqualTo(sublen);
+ current = verifyCounterStatisticValue(statistics, STREAM_READ_BYTES,
+ current + sublen);
+ Assertions.assertThat(in.read(pos, buf128, 0, bufferLen))
+ .describedAs("Read(buffer) overlapping EOF")
+ .isEqualTo(sublen);
+ verifyCounterStatisticValue(statistics, STREAM_READ_BYTES,
+ current + sublen);
+ }
+ } finally {
+ fs.delete(path, false);
+ }
+ }
+
+ /**
+ * Verify the bytes read value, taking into account block size.
+ * @param statistics stats
+ * @param current current count
+ * @param bytesRead bytes explicitly read
+ * @param bufferSize buffer size of stream
+ * @return the current count of bytes read ignoring block size
+ */
+ public long verifyBytesRead(final IOStatistics statistics,
+ final long current,
+ final int bytesRead, final int bufferSize) {
+ // final position. for unbuffered read, this is the expected value
+ long finalPos = current + bytesRead;
+ long expected = finalPos;
+ if (bufferSize > 0) {
+ // buffered. count of read is number of buffers already read
+ // plus the current buffer, multiplied by that buffer size
+ expected = bufferSize * (1 + (current / bufferSize));
+ }
+ verifyCounterStatisticValue(statistics, STREAM_READ_BYTES, expected);
+ return finalPos;
+ }
+
+ /**
+ * Buffer size for reads.
+ * Filesystems performing block reads (checksum, etc)
+ * must return their buffer value is
+ * @return buffer capacity; 0 for unbuffered
+ */
+ public int readBufferSize() {
+ return 0;
+ }
+
+ /**
+ * Keys which the output stream must support.
+ * @return a list of keys
+ */
+ public List outputStreamStatisticKeys() {
+ return Collections.singletonList(STREAM_WRITE_BYTES);
+ }
+
+ /**
+ * Keys which the input stream must support.
+ * @return a list of keys
+ */
+ public List inputStreamStatisticKeys() {
+ return Collections.singletonList(STREAM_READ_BYTES);
+ }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractStreamIOStatistics.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractStreamIOStatistics.java
new file mode 100644
index 0000000000000..642baec502d2e
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractStreamIOStatistics.java
@@ -0,0 +1,80 @@
+/*
+ * 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.contract.localfs;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractStreamIOStatisticsTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BYTES;
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_EXCEPTIONS;
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS;
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_SKIP_BYTES;
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_SKIP_OPERATIONS;
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_WRITE_BYTES;
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_WRITE_EXCEPTIONS;
+
+/**
+ * Test IOStatistics through the local FS.
+ */
+public class TestLocalFSContractStreamIOStatistics extends
+ AbstractContractStreamIOStatisticsTest {
+
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new LocalFSContract(conf);
+ }
+
+ /**
+ * Keys which the input stream must support.
+ * @return a list of keys
+ */
+ public List inputStreamStatisticKeys() {
+ return Arrays.asList(STREAM_READ_BYTES,
+ STREAM_READ_EXCEPTIONS,
+ STREAM_READ_SEEK_OPERATIONS,
+ STREAM_READ_SKIP_OPERATIONS,
+ STREAM_READ_SKIP_BYTES);
+ }
+
+ /**
+ * Keys which the output stream must support.
+ * @return a list of keys
+ */
+ @Override
+ public List outputStreamStatisticKeys() {
+ return Arrays.asList(STREAM_WRITE_BYTES,
+ STREAM_WRITE_EXCEPTIONS);
+ }
+
+ @Override
+ public int readBufferSize() {
+ return 1024;
+ }
+
+ @Override
+ public boolean streamWritesInBlocks() {
+ return true;
+ }
+
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/IOStatisticAssertions.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/IOStatisticAssertions.java
new file mode 100644
index 0000000000000..4fa6b227ba125
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/IOStatisticAssertions.java
@@ -0,0 +1,466 @@
+/*
+ * 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.statistics;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.util.Map;
+
+import org.assertj.core.api.AbstractLongAssert;
+import org.assertj.core.api.ObjectAssert;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Assertions and any other support for IOStatistics testing.
+ *
+ * If used downstream: know it is unstable.
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public final class IOStatisticAssertions {
+
+ private static final String COUNTER = "Counter";
+
+ private static final String GAUGE = "Gauge";
+
+ private static final String MINIMUM = "Minimum";
+
+ private static final String MAXIMUM = "Maxiumum";
+
+ private static final String MEAN = "Mean";
+
+ private IOStatisticAssertions() {
+ }
+
+ /**
+ * Get a required counter statistic.
+ * @param stats statistics source
+ * @param key statistic key
+ * @return the value
+ */
+ public static long lookupCounterStatistic(
+ final IOStatistics stats,
+ final String key) {
+ return lookupStatistic(COUNTER, key, stats.counters());
+ }
+
+ /**
+ * Get a required gauge statistic.
+ * @param stats statistics source
+ * @param key statistic key
+ * @return the value
+ */
+ public static long lookupGaugeStatistic(
+ final IOStatistics stats,
+ final String key) {
+ return lookupStatistic(GAUGE, key, stats.gauges());
+ }
+
+ /**
+ * Get a required maximum statistic.
+ * @param stats statistics source
+ * @param key statistic key
+ * @return the value
+ */
+ public static long lookupMaximumStatistic(
+ final IOStatistics stats,
+ final String key) {
+ return lookupStatistic(MAXIMUM, key, stats.maximums());
+ }
+
+ /**
+ * Get a required minimum statistic.
+ * @param stats statistics source
+ * @param key statistic key
+ * @return the value
+ */
+ public static long lookupMinimumStatistic(
+ final IOStatistics stats,
+ final String key) {
+ return lookupStatistic(MINIMUM, key, stats.minimums());
+ }
+
+ /**
+ * Get a required mean statistic.
+ * @param stats statistics source
+ * @param key statistic key
+ * @return the value
+ */
+ public static MeanStatistic lookupMeanStatistic(
+ final IOStatistics stats,
+ final String key) {
+ return lookupStatistic(MEAN, key, stats.meanStatistics());
+ }
+
+ /**
+ * Get a required counter statistic.
+ * @param type of map element
+ * @param type type for error text
+ * @param key statistic key
+ * @param map map to probe
+ * @return the value
+ */
+ private static E lookupStatistic(
+ final String type,
+ final String key,
+ final Map map) {
+ final E statistic = map.get(key);
+ assertThat(statistic)
+ .describedAs("%s named %s", type, key)
+ .isNotNull();
+ return statistic;
+ }
+
+ /**
+ * Assert that a counter has an expected value.
+ * @param stats statistics source
+ * @param key statistic key
+ * @param value expected value.
+ * @return the value (which always equals the expected value)
+ */
+ public static long verifyCounterStatisticValue(
+ final IOStatistics stats,
+ final String key,
+ final long value) {
+ return verifyStatisticValue(COUNTER, key, stats.counters(), value);
+ }
+
+ /**
+ * Assert that a gauge has an expected value.
+ * @param stats statistics source
+ * @param key statistic key
+ * @param value expected value.
+ * @return the value (which always equals the expected value)
+ */
+ public static long verifyGaugeStatisticValue(
+ final IOStatistics stats,
+ final String key,
+ final long value) {
+ return verifyStatisticValue(GAUGE, key, stats.gauges(), value);
+ }
+
+ /**
+ * Assert that a maximum has an expected value.
+ * @param stats statistics source
+ * @param key statistic key
+ * @param value expected value.
+ * @return the value (which always equals the expected value)
+ */
+ public static long verifyMaximumStatisticValue(
+ final IOStatistics stats,
+ final String key,
+ final long value) {
+ return verifyStatisticValue(MAXIMUM, key, stats.maximums(), value);
+ }
+
+ /**
+ * Assert that a minimum has an expected value.
+ * @param stats statistics source
+ * @param key statistic key
+ * @param value expected value.
+ * @return the value (which always equals the expected value)
+ */
+ public static long verifyMinimumStatisticValue(
+ final IOStatistics stats,
+ final String key,
+ final long value) {
+ return verifyStatisticValue(MINIMUM, key, stats.minimums(), value);
+ }
+
+ /**
+ * Assert that a mean has an expected value.
+ * @param stats statistics source
+ * @param key statistic key
+ * @param value expected value.
+ * @return the value (which always equals the expected value)
+ */
+ public static MeanStatistic verifyMeanStatisticValue(
+ final IOStatistics stats,
+ final String key,
+ final MeanStatistic value) {
+ return verifyStatisticValue(MEAN, key, stats.meanStatistics(), value);
+ }
+
+ /**
+ * Assert that a given statistic has an expected value.
+ * @param type type for error text
+ * @param key statistic key
+ * @param map map to look up
+ * @param value expected value.
+ * @param type of map element
+ * @return the value (which always equals the expected value)
+ */
+ private static E verifyStatisticValue(
+ final String type,
+ final String key,
+ final Map map,
+ final E value) {
+ final E statistic = lookupStatistic(type, key, map);
+ assertThat(statistic)
+ .describedAs("%s named %s with expected value %s", type,
+ key, value)
+ .isEqualTo(value);
+ return statistic;
+ }
+
+
+ /**
+ * Assert that a given statistic has an expected value.
+ * @param type of map element
+ * @param type type for error text
+ * @param key statistic key
+ * @param map map to look up
+ * @return an ongoing assertion
+ */
+ private static ObjectAssert assertThatStatistic(
+ final String type,
+ final String key,
+ final Map map) {
+ final E statistic = lookupStatistic(type, key, map);
+ return assertThat(statistic)
+ .describedAs("%s named %s", type, key);
+ }
+
+ /**
+ * Assert that a given statistic has an expected value.
+ * @param type of map element
+ * @param type type for error text
+ * @param key statistic key
+ * @param map map to look up
+ * @return an ongoing assertion
+ */
+ private static AbstractLongAssert> assertThatLongStatistic(
+ final String type,
+ final String key,
+ final Map map) {
+ final long statistic = lookupStatistic(type, key, map);
+ return assertThat(statistic)
+ .describedAs("%s named %s", type, key);
+ }
+
+ /**
+ * Start an assertion chain on
+ * a required counter statistic.
+ * @param stats statistics source
+ * @param key statistic key
+ * @return an ongoing assertion
+ */
+ public static AbstractLongAssert> assertThatCounterStatistic(
+ final IOStatistics stats,
+ final String key) {
+ return assertThatLongStatistic(COUNTER, key, stats.counters());
+ }
+
+ /**
+ * Start an assertion chain on
+ * a required gauge statistic.
+ * @param stats statistics source
+ * @param key statistic key
+ * @return an ongoing assertion
+ */
+ public static AbstractLongAssert> assertThatGaugeStatistic(
+ final IOStatistics stats,
+ final String key) {
+ return assertThatLongStatistic(GAUGE, key, stats.gauges());
+ }
+
+ /**
+ * Start an assertion chain on
+ * a required minimum statistic.
+ * @param stats statistics source
+ * @param key statistic key
+ * @return an ongoing assertion
+ */
+ public static AbstractLongAssert> assertThatMinimumStatistic(
+ final IOStatistics stats,
+ final String key) {
+ return assertThatLongStatistic(MINIMUM, key, stats.minimums());
+ }
+
+ /**
+ * Start an assertion chain on
+ * a required maximum statistic.
+ * @param stats statistics source
+ * @param key statistic key
+ * @return an ongoing assertion
+ */
+ public static AbstractLongAssert> assertThatMaximumStatistic(
+ final IOStatistics stats,
+ final String key) {
+ return assertThatLongStatistic(MAXIMUM, key, stats.maximums());
+ }
+
+ /**
+ * Start an assertion chain on
+ * a required mean statistic.
+ * @param stats statistics source
+ * @param key statistic key
+ * @return an ongoing assertion
+ */
+ public static ObjectAssert assertThatMeanStatistic(
+ final IOStatistics stats,
+ final String key) {
+ return assertThatStatistic(MEAN, key, stats.meanStatistics());
+ }
+
+ /**
+ * Start an assertion chain on
+ * a required mean statistic with the initial validation on the
+ * sample count and sum.
+ * @param stats statistics source
+ * @param key statistic key
+ * @return an ongoing assertion
+ */
+ public static ObjectAssert assertThatMeanStatisticMatches(
+ final IOStatistics stats,
+ final String key,
+ final long samples,
+ final long sum) {
+ return assertThatMeanStatistic(stats, key)
+ .matches(p -> (p.getSamples() == samples),
+ "samples == " + samples)
+ .matches(p -> (p.getSum() == sum),
+ "sum == " + sum);
+ }
+
+ /**
+ * Assert that a given counter statistic is untracked.
+ * @param stats statistics source
+ * @param type type for error text
+ * @param key statistic key
+ * @param map map to probe
+ */
+ private static void assertUntracked(final IOStatistics stats,
+ final String type,
+ final String key,
+ final Map map) {
+ assertThat(map.containsKey(key))
+ .describedAs("%s %s is tracked in %s", type, key, stats)
+ .isFalse();
+ }
+
+ /**
+ * Assert that a given counter statistic is untracked.
+ * @param stats statistics source
+ * @param type type for error text
+ * @param key statistic key
+ * @param map map to probe
+ */
+ private static void assertTracked(final IOStatistics stats,
+ final String type,
+ final String key,
+ final Map map) {
+ assertThat(map.containsKey(key))
+ .describedAs("%s %s is not tracked in %s", type, key, stats)
+ .isTrue();
+ }
+
+ /**
+ * Assert that a given statistic is tracked.
+ * @param stats statistics source
+ * @param key statistic key
+ */
+ public static void assertCounterStatisticIsTracked(
+ final IOStatistics stats,
+ final String key) {
+ assertTracked(stats, COUNTER, key, stats.counters());
+ }
+
+ /**
+ * Assert that a given counter statistic is untracked.
+ * @param stats statistics source
+ * @param key statistic key
+ */
+ public static void assertCounterStatisticIsUntracked(
+ final IOStatistics stats,
+ final String key) {
+ assertUntracked(stats, COUNTER, key, stats.counters());
+ }
+
+ /**
+ * Assert that an object is a statistics source and that the
+ * statistics is not null.
+ * @param source source object.
+ */
+ public static void assertIsStatisticsSource(Object source) {
+ assertThat(source)
+ .describedAs("Object %s", source)
+ .isInstanceOf(IOStatisticsSource.class)
+ .extracting(o -> ((IOStatisticsSource) o).getIOStatistics())
+ .isNotNull();
+ }
+
+ /**
+ * Query the source for the statistics; fails if the statistics
+ * returned are null or the class does not implement the API.
+ * @param source source object.
+ * @return the statistics it provides.
+ */
+ public static IOStatistics extractStatistics(Object source) {
+ assertThat(source)
+ .describedAs("Object %s", source)
+ .isInstanceOf(IOStatisticsSource.class);
+ IOStatisticsSource ios = (IOStatisticsSource) source;
+ return extractStatistics(ios);
+ }
+
+ /**
+ * Get the non-null statistics.
+ * @param ioStatisticsSource source
+ * @return the statistics, guaranteed to be non null
+ */
+ private static IOStatistics extractStatistics(
+ final IOStatisticsSource ioStatisticsSource) {
+ IOStatistics statistics = ioStatisticsSource.getIOStatistics();
+ assertThat(statistics)
+ .describedAs("Statistics from %s", ioStatisticsSource)
+ .isNotNull();
+ return statistics;
+ }
+
+ /**
+ * Perform a serialization round trip on a statistics instance.
+ * @param stat statistic
+ * @return the deserialized version.
+ */
+ public static IOStatistics roundTrip(final IOStatistics stat)
+ throws IOException, ClassNotFoundException {
+ assertThat(stat).isInstanceOf(Serializable.class);
+ ByteArrayOutputStream baos = new ByteArrayOutputStream(1024);
+ try (ObjectOutputStream oos = new ObjectOutputStream(baos)) {
+ oos.writeObject(stat);
+ }
+ ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
+ IOStatistics deser;
+ try (ObjectInputStream ois = new ObjectInputStream(bais)) {
+ deser = (IOStatistics) ois.readObject();
+ }
+ return deser;
+ }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestDynamicIOStatistics.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestDynamicIOStatistics.java
new file mode 100644
index 0000000000000..0efa2b2a1fb82
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestDynamicIOStatistics.java
@@ -0,0 +1,311 @@
+/*
+ * 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.statistics;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.statistics.impl.SourceWrappedStatistics;
+import org.apache.hadoop.metrics2.MetricsInfo;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.test.AbstractHadoopTestBase;
+
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertCounterStatisticIsTracked;
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertCounterStatisticIsUntracked;
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyCounterStatisticValue;
+import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatistics;
+import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatisticsSource;
+import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToString;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.ENTRY_PATTERN;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.NULL_SOURCE;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.dynamicIOStatistics;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.emptyStatistics;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/**
+ * verify dynamic statistics are dynamic, except when you iterate through
+ * them, along with other tests of the class's behavior.
+ */
+public class TestDynamicIOStatistics extends AbstractHadoopTestBase {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(TestDynamicIOStatistics.class);
+
+ private static final String ALONG = "along";
+
+ private static final String AINT = "aint";
+
+ private static final String COUNT = "count";
+
+ private static final String EVAL = "eval";
+
+ /**
+ * The statistics.
+ */
+ private IOStatistics statistics = emptyStatistics();
+
+ /**
+ * A source of these statistics.
+ */
+ private IOStatisticsSource statsSource;
+
+ private final AtomicLong aLong = new AtomicLong();
+
+ private final AtomicInteger aInt = new AtomicInteger();
+
+ private final MutableCounterLong counter = new MutableCounterLong(
+ new Info("counter"), 0);
+
+ private long evalLong;
+
+ private static final String[] KEYS = new String[]{ALONG, AINT, COUNT, EVAL};
+
+ @Before
+ public void setUp() throws Exception {
+ statistics = dynamicIOStatistics()
+ .withAtomicLongCounter(ALONG, aLong)
+ .withAtomicIntegerCounter(AINT, aInt)
+ .withMutableCounter(COUNT, counter)
+ .withLongFunctionCounter(EVAL, x -> evalLong)
+ .build();
+ statsSource = new SourceWrappedStatistics(statistics);
+ }
+
+ /**
+ * The eval operation is foundational.
+ */
+ @Test
+ public void testEval() throws Throwable {
+ verifyCounterStatisticValue(statistics, EVAL, 0);
+ evalLong = 10;
+ verifyCounterStatisticValue(statistics, EVAL, 10);
+ }
+
+ /**
+ * Atomic Long statistic.
+ */
+ @Test
+ public void testAlong() throws Throwable {
+ verifyCounterStatisticValue(statistics, ALONG, 0);
+ aLong.addAndGet(1);
+ verifyCounterStatisticValue(statistics, ALONG, 1);
+ }
+
+ /**
+ * Atomic Int statistic.
+ */
+ @Test
+ public void testAint() throws Throwable {
+ verifyCounterStatisticValue(statistics, AINT, 0);
+ aInt.addAndGet(1);
+ verifyCounterStatisticValue(statistics, AINT, 1);
+ }
+
+ /**
+ * Metrics2 counter.
+ */
+ @Test
+ public void testCounter() throws Throwable {
+ verifyCounterStatisticValue(statistics, COUNT, 0);
+ counter.incr();
+ verifyCounterStatisticValue(statistics, COUNT, 1);
+ }
+
+ /**
+ * keys() returns all the keys.
+ */
+ @Test
+ public void testKeys() throws Throwable {
+ Assertions.assertThat(statistics.counters().keySet())
+ .describedAs("statistic keys of %s", statistics)
+ .containsExactlyInAnyOrder(KEYS);
+ }
+
+ @Test
+ public void testIteratorHasAllKeys() throws Throwable {
+ // go through the statistics iterator and assert that it contains exactly
+ // the values.
+ assertThat(statistics.counters().keySet())
+ .containsExactlyInAnyOrder(KEYS);
+ }
+
+ /**
+ * Verify that the iterator is taken from
+ * a snapshot of the values.
+ */
+ @Test
+ public void testIteratorIsSnapshot() throws Throwable {
+ // set the counters all to 1
+ incrementAllCounters();
+ // take the snapshot
+ final Iterator> it =
+ statistics.counters().entrySet().iterator();
+ // increment the counters
+ incrementAllCounters();
+ // now assert that all the iterator values are of value 1
+ while (it.hasNext()) {
+ Map.Entry next = it.next();
+ assertThat(next.getValue())
+ .describedAs("Value of entry %s", next)
+ .isEqualTo(1);
+ }
+ }
+
+ @Test
+ public void testUnknownStatistic() throws Throwable {
+ assertCounterStatisticIsUntracked(statistics, "anything");
+ }
+
+ @Test
+ public void testStatisticsTrackedAssertion() throws Throwable {
+ // expect an exception to be raised when an assertion
+ // is made that an unknown statistic is tracked,.
+ assertThatThrownBy(() ->
+ assertCounterStatisticIsTracked(statistics, "anything"))
+ .isInstanceOf(AssertionError.class);
+ }
+
+ @Test
+ public void testStatisticsValueAssertion() throws Throwable {
+ // expect an exception to be raised when
+ // an assertion is made about the value of an unknown statistics
+ assertThatThrownBy(() ->
+ verifyCounterStatisticValue(statistics, "anything", 0))
+ .isInstanceOf(AssertionError.class);
+ }
+
+ /**
+ * Serialization round trip will preserve all the values.
+ */
+ @Test
+ public void testSerDeser() throws Throwable {
+ incrementAllCounters();
+ IOStatistics stat = IOStatisticsSupport.snapshotIOStatistics(statistics);
+ incrementAllCounters();
+ IOStatistics deser = IOStatisticAssertions.roundTrip(stat);
+ assertThat(deser.counters().keySet())
+ .containsExactlyInAnyOrder(KEYS);
+ for (Map.Entry e : deser.counters().entrySet()) {
+ assertThat(e.getValue())
+ .describedAs("Value of entry %s", e)
+ .isEqualTo(1);
+ }
+ }
+
+ @Test
+ public void testStringification() throws Throwable {
+ assertThat(ioStatisticsToString(statistics))
+ .isNotBlank()
+ .contains(KEYS);
+ }
+
+ @Test
+ public void testDemandStringification() throws Throwable {
+ String counterPattern = ENTRY_PATTERN;
+ // this is not yet evaluated
+ Object demand = demandStringifyIOStatistics(statistics);
+ // nor is this.
+ Object demandSource = demandStringifyIOStatisticsSource(statsSource);
+
+ // show it evaluates
+ String formatted1 = String.format(counterPattern, ALONG, aLong.get());
+ assertThat(demand
+ .toString())
+ .contains(formatted1);
+ assertThat(demandSource
+ .toString())
+ .contains(formatted1);
+
+ // when the counters are incremented
+ incrementAllCounters();
+ incrementAllCounters();
+ // there are new values to expect
+ String formatted2 = String.format(counterPattern, ALONG, aLong.get());
+ assertThat(demand
+ .toString())
+ .doesNotContain(formatted1)
+ .contains(formatted2);
+ assertThat(demandSource
+ .toString())
+ .doesNotContain(formatted1)
+ .contains(formatted2);
+ }
+
+ @Test
+ public void testNullSourceStringification() throws Throwable {
+ assertThat(demandStringifyIOStatisticsSource((IOStatisticsSource) null)
+ .toString())
+ .isEqualTo(NULL_SOURCE);
+ }
+
+ @Test
+ public void testNullStatStringification() throws Throwable {
+ assertThat(demandStringifyIOStatistics((IOStatistics) null)
+ .toString())
+ .isEqualTo(NULL_SOURCE);
+ }
+
+ @Test
+ public void testStringLogging() throws Throwable {
+ LOG.info("Output {}", demandStringifyIOStatistics(statistics));
+ }
+
+ /**
+ * Increment all the counters from their current value.
+ */
+ private void incrementAllCounters() {
+ aLong.incrementAndGet();
+ aInt.incrementAndGet();
+ evalLong += 1;
+ counter.incr();
+ }
+
+ /**
+ * Needed to provide a metrics info instance for the counter
+ * constructor.
+ */
+ private static final class Info implements MetricsInfo {
+
+ private final String name;
+
+ private Info(final String name) {
+ this.name = name;
+ }
+
+ @Override
+ public String name() {
+ return name;
+ }
+
+ @Override
+ public String description() {
+ return name;
+ }
+ }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestEmptyIOStatistics.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestEmptyIOStatistics.java
new file mode 100644
index 0000000000000..a4a64c2bf1952
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestEmptyIOStatistics.java
@@ -0,0 +1,110 @@
+/*
+ * 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.statistics;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding;
+import org.apache.hadoop.test.AbstractHadoopTestBase;
+
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertCounterStatisticIsTracked;
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertCounterStatisticIsUntracked;
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyCounterStatisticValue;
+import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToString;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.emptyStatistics;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/**
+ * Test handling of the empty IO statistics class.
+ */
+public class TestEmptyIOStatistics extends AbstractHadoopTestBase {
+
+ private final IOStatistics empty = emptyStatistics();
+
+ @Test
+ public void testUnknownStatistic() throws Throwable {
+ assertCounterStatisticIsUntracked(empty, "anything");
+ }
+
+ @Test
+ public void testStatisticsTrackedAssertion() throws Throwable {
+ // expect an exception to be raised when an assertion
+ // is made that an unknown statistic is tracked,.
+ assertThatThrownBy(() ->
+ assertCounterStatisticIsTracked(empty, "anything"))
+ .isInstanceOf(AssertionError.class);
+ }
+
+ @Test
+ public void testStatisticsValueAssertion() throws Throwable {
+ // expect an exception to be raised when
+ // an assertion is made about the value of an unknown statistics
+ assertThatThrownBy(() ->
+ verifyCounterStatisticValue(empty, "anything", 0))
+ .isInstanceOf(AssertionError.class);
+ }
+
+ @Test
+ public void testEmptySnapshot() throws Throwable {
+ final IOStatistics stat = IOStatisticsSupport.snapshotIOStatistics(empty);
+ assertThat(stat.counters().keySet())
+ .describedAs("keys of snapshot")
+ .isEmpty();
+ IOStatistics deser = IOStatisticAssertions.roundTrip(stat);
+ assertThat(deser.counters().keySet())
+ .describedAs("keys of deserialized snapshot")
+ .isEmpty();
+ }
+
+ @Test
+ public void testStringification() throws Throwable {
+ assertThat(ioStatisticsToString(empty))
+ .isNotBlank();
+ }
+
+ @Test
+ public void testWrap() throws Throwable {
+ IOStatisticsSource statisticsSource = IOStatisticsBinding.wrap(empty);
+ assertThat(statisticsSource.getIOStatistics())
+ .isSameAs(empty);
+ }
+
+ @Test
+ public void testStringifyNullSource() throws Throwable {
+ assertThat(IOStatisticsLogging.ioStatisticsSourceToString(null))
+ .isEmpty();
+ }
+
+ @Test
+ public void testStringifyNullStats() throws Throwable {
+ assertThat(
+ IOStatisticsLogging.ioStatisticsSourceToString(
+ IOStatisticsBinding.wrap(null)))
+ .isEmpty();
+ }
+
+ @Test
+ public void testStringificationNull() throws Throwable {
+ assertThat(ioStatisticsToString(null))
+ .describedAs("Null statistics should stringify to \"\"")
+ .isEmpty();
+ }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsSnapshot.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsSnapshot.java
new file mode 100644
index 0000000000000..19694e28d754b
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsSnapshot.java
@@ -0,0 +1,125 @@
+/*
+ * 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.statistics;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding;
+import org.apache.hadoop.test.AbstractHadoopTestBase;
+import org.apache.hadoop.util.JsonSerialization;
+
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.*;
+import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToString;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/**
+ * Test handling of the {@link IOStatisticsSnapshot} class.
+ */
+public class TestIOStatisticsSnapshot extends AbstractHadoopTestBase {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(TestIOStatisticsSnapshot.class);
+
+ private final IOStatisticsSnapshot snapshot = new IOStatisticsSnapshot();
+
+ private MeanStatistic mean1;
+
+ private MeanStatistic mean0;
+
+ @Before
+ public void setup() throws Exception {
+ snapshot.counters().put("c1", 0L);
+ snapshot.gauges().put("g1", 1L);
+ snapshot.minimums().put("m1", -1L);
+ mean1 = new MeanStatistic(1, 1);
+ snapshot.meanStatistics().put("mean1",
+ mean1);
+ mean0 = new MeanStatistic(0, 1);
+ snapshot.meanStatistics().put("mean0",
+ mean0);
+ }
+
+ @Test
+ public void testTrackedValues() throws Throwable {
+ verifyCounterStatisticValue(snapshot, "c1", 0L);
+ verifyGaugeStatisticValue(snapshot, "g1", 1L);
+ verifyMinimumStatisticValue(snapshot, "m1", -1L);
+ verifyMeanStatisticValue(snapshot, "mean0",
+ new MeanStatistic(0, 1));
+ }
+
+
+ @Test
+ public void testStatisticsValueAssertion() throws Throwable {
+ // expect an exception to be raised when
+ // an assertion is made about the value of an unknown statistics
+ assertThatThrownBy(() ->
+ verifyCounterStatisticValue(snapshot, "anything", 0))
+ .isInstanceOf(AssertionError.class);
+ }
+
+ @Test
+ public void testStringification() throws Throwable {
+ assertThat(ioStatisticsToString(snapshot))
+ .isNotBlank();
+ }
+
+ @Test
+ public void testStringification2() throws Throwable {
+
+ String ss = snapshot.toString();
+ LOG.info("original {}", ss);
+ Assertions.assertThat(ss)
+ .describedAs("snapshot toString()")
+ .contains("c1=0")
+ .contains("g1=1");
+ }
+
+ @Test
+ public void testWrap() throws Throwable {
+ IOStatisticsSource statisticsSource = IOStatisticsBinding.wrap(snapshot);
+ assertThat(statisticsSource.getIOStatistics())
+ .isSameAs(snapshot);
+ }
+
+ @Test
+ public void testRoundTrip() throws Throwable {
+ JsonSerialization serializer
+ = IOStatisticsSnapshot.serializer();
+
+ String json = serializer.toJson(snapshot);
+ LOG.info("serialized form\n{}", json);
+ IOStatisticsSnapshot deser = serializer.fromJson(json);
+ LOG.info("deserialized {}", deser);
+ verifyCounterStatisticValue(deser, "c1", 0L);
+ verifyGaugeStatisticValue(deser, "g1", 1L);
+ verifyMinimumStatisticValue(deser, "m1", -1L);
+ verifyMeanStatisticValue(deser, "mean0",
+ new MeanStatistic(0, 1));
+ verifyMeanStatisticValue(deser, "mean1",
+ snapshot.meanStatistics().get("mean1"));
+ }
+
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsStore.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsStore.java
new file mode 100644
index 0000000000000..cccf2744df980
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsStore.java
@@ -0,0 +1,189 @@
+/*
+ * 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.statistics;
+
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
+import org.apache.hadoop.test.AbstractHadoopTestBase;
+import org.apache.hadoop.util.JsonSerialization;
+
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatMaximumStatistic;
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatMeanStatistic;
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatMeanStatisticMatches;
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatMinimumStatistic;
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyCounterStatisticValue;
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyGaugeStatisticValue;
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyMaximumStatisticValue;
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyMinimumStatisticValue;
+import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore;
+
+public class TestIOStatisticsStore extends AbstractHadoopTestBase {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(TestIOStatisticsStore.class);
+
+
+ private static final String COUNT = "count";
+
+ private static final String GAUGE = "gauge";
+
+ private static final String MIN = "min";
+
+ private static final String MAX = "max";
+
+ private static final String MEAN = "mean";
+
+ public static final String UNKNOWN = "unknown";
+
+ private IOStatisticsStore stats;
+
+ @Before
+ public void setup() {
+ stats = iostatisticsStore()
+ .withCounters(COUNT)
+ .withGauges(GAUGE)
+ .withMinimums(MIN)
+ .withMaximums(MAX)
+ .withMeanStatistics(MEAN)
+ .withDurationTracking(OBJECT_LIST_REQUEST)
+ .build();
+ }
+
+ @After
+ public void teardown() {
+ LOG.info("stats {}", stats);
+ }
+
+ /**
+ * Gauges go up and down.
+ */
+ @Test
+ public void testGauges() throws Throwable {
+ stats.setGauge(GAUGE, 1);
+ verifyGaugeStatisticValue(stats, GAUGE, 1);
+ stats.incrementGauge(GAUGE, 1);
+ verifyGaugeStatisticValue(stats, GAUGE, 2);
+ stats.setGauge(GAUGE, -1);
+ verifyGaugeStatisticValue(stats, GAUGE, -1);
+ stats.incrementGauge(GAUGE, -1);
+ verifyGaugeStatisticValue(stats, GAUGE, -2);
+ Assertions.assertThat(stats.getGaugeReference(GAUGE).get())
+ .isEqualTo(-2);
+ stats.incrementGauge(UNKNOWN, 1);
+ stats.setGauge(UNKNOWN, 1);
+ }
+
+ @Test
+ public void testMinimums() throws Throwable {
+ stats.setMinimum(MIN, 100);
+ verifyMinimumStatisticValue(stats, MIN, 100);
+ stats.setMinimum(MIN, 100);
+ // will do nothing as it is higher
+ stats.addMinimumSample(MIN, 200);
+ verifyMinimumStatisticValue(stats, MIN, 100);
+ stats.addMinimumSample(MIN, 10);
+ verifyMinimumStatisticValue(stats, MIN, 10);
+ stats.setMinimum(UNKNOWN, 100);
+ stats.addMinimumSample(UNKNOWN, 200);
+ }
+
+ @Test
+ public void testMaximums() throws Throwable {
+ stats.setMaximum(MAX, 100);
+ verifyMaximumStatisticValue(stats, MAX, 100);
+ stats.setMaximum(MAX, 100);
+ stats.addMaximumSample(MAX, 200);
+ verifyMaximumStatisticValue(stats, MAX, 200);
+ stats.addMaximumSample(MAX, 10);
+ verifyMaximumStatisticValue(stats, MAX, 200);
+ stats.setMaximum(UNKNOWN, 100);
+ stats.addMaximumSample(UNKNOWN, 200);
+ }
+
+ @Test
+ public void testMeans() throws Throwable {
+ stats.setMeanStatistic(MEAN,
+ new MeanStatistic(1, 1));
+
+ assertThatMeanStatisticMatches(stats, MEAN, 1, 1)
+ .matches(p -> p.mean() == 1, "mean");
+ stats.addMeanStatisticSample(MEAN, 9);
+ assertThatMeanStatisticMatches(stats, MEAN, 2, 10)
+ .matches(p -> p.mean() == 5, "mean");
+ }
+
+ @Test
+ public void testRoundTrip() throws Throwable {
+ JsonSerialization serializer
+ = IOStatisticsSnapshot.serializer();
+ stats.incrementCounter(COUNT, 1);
+ stats.setGauge(GAUGE, -1);
+ stats.addMaximumSample(MAX, 200);
+ stats.addMinimumSample(MIN, -100);
+ stats.addMeanStatisticSample(MEAN, 1);
+ stats.addMeanStatisticSample(MEAN, 9);
+
+ String json = serializer.toJson(snapshotIOStatistics(stats));
+ LOG.info("serialized form\n{}", json);
+ IOStatisticsSnapshot deser = serializer.fromJson(json);
+ LOG.info("deserialized {}", deser);
+ verifyCounterStatisticValue(deser, COUNT, 1L);
+ verifyGaugeStatisticValue(deser, GAUGE, -1);
+ verifyMaximumStatisticValue(deser, MAX, 200);
+ verifyMinimumStatisticValue(deser, MIN, -100);
+ assertThatMeanStatisticMatches(deser, MEAN, 2, 10)
+ .matches(p -> p.mean() == 5, "mean");
+
+ }
+
+ /**
+ * Duration tracking.
+ */
+ @Test
+ public void testDuration() throws Throwable {
+ DurationTracker tracker =
+ stats.trackDuration(OBJECT_LIST_REQUEST);
+ verifyCounterStatisticValue(stats, OBJECT_LIST_REQUEST, 1L);
+ Thread.sleep(1000);
+ tracker.close();
+ try (DurationTracker ignored =
+ stats.trackDuration(OBJECT_LIST_REQUEST)) {
+ Thread.sleep(1000);
+ }
+ LOG.info("Statistics: {}", stats);
+ verifyCounterStatisticValue(stats, OBJECT_LIST_REQUEST, 2L);
+ assertThatMinimumStatistic(stats, OBJECT_LIST_REQUEST + ".min")
+ .isGreaterThan(0);
+ assertThatMaximumStatistic(stats, OBJECT_LIST_REQUEST + ".max")
+ .isGreaterThan(0);
+ assertThatMeanStatistic(stats, OBJECT_LIST_REQUEST + ".mean")
+ .hasFieldOrPropertyWithValue("samples", 2L)
+ .matches(s -> s.getSum() > 0)
+ .matches(s -> s.mean() > 0.0);
+ }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestMeanStatistic.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestMeanStatistic.java
new file mode 100644
index 0000000000000..83b096fc37713
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestMeanStatistic.java
@@ -0,0 +1,216 @@
+/*
+ * 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.statistics;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.test.AbstractHadoopTestBase;
+import org.apache.hadoop.util.JsonSerialization;
+
+public class TestMeanStatistic extends AbstractHadoopTestBase {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(TestMeanStatistic.class);
+
+ private static final int TEN = 10;
+
+ private static final double ZEROD = 0.0d;
+
+ private static final double TEND = 10.0d;
+
+ private final MeanStatistic empty = new MeanStatistic(0, 0);
+
+ private final MeanStatistic tenFromOne = new MeanStatistic(1, TEN);
+
+ private final MeanStatistic tenFromTen = new MeanStatistic(TEN, TEN);
+
+ @Test
+ public void testEmptiness() throws Throwable {
+ Assertions.assertThat(empty)
+ .matches(MeanStatistic::isEmpty, "is empty")
+ .isEqualTo(new MeanStatistic(0, TEN))
+ .isEqualTo(new MeanStatistic())
+ .isNotEqualTo(tenFromOne);
+ Assertions.assertThat(empty.mean())
+ .isEqualTo(ZEROD);
+ Assertions.assertThat(empty.toString())
+ .contains("0.0");
+ }
+
+ @Test
+ public void testTenFromOne() throws Throwable {
+ Assertions.assertThat(tenFromOne)
+ .matches(p -> !p.isEmpty(), "is not empty")
+ .isEqualTo(tenFromOne)
+ .isNotEqualTo(tenFromTen);
+ Assertions.assertThat(tenFromOne.mean())
+ .isEqualTo(TEND);
+ }
+
+ @Test
+ public void testNegativeSamplesAreEmpty() throws Throwable {
+ MeanStatistic stat = new MeanStatistic(-10, 1);
+ Assertions.assertThat(stat)
+ .describedAs("stat with negative samples")
+ .matches(MeanStatistic::isEmpty, "is empty")
+ .isEqualTo(empty)
+ .extracting(MeanStatistic::mean)
+ .isEqualTo(ZEROD);
+ Assertions.assertThat(stat.toString())
+ .contains("0.0");
+
+ }
+
+ @Test
+ public void testCopyNonEmpty() throws Throwable {
+ MeanStatistic stat = tenFromOne.copy();
+ Assertions.assertThat(stat)
+ .describedAs("copy of " + tenFromOne)
+ .isEqualTo(tenFromOne)
+ .isNotSameAs(tenFromOne);
+ }
+
+ @Test
+ public void testCopyEmpty() throws Throwable {
+ MeanStatistic stat = empty.copy();
+ Assertions.assertThat(stat)
+ .describedAs("copy of " + empty)
+ .isEqualTo(empty)
+ .isNotSameAs(empty);
+ }
+
+ @Test
+ public void testDoubleSamples() throws Throwable {
+ MeanStatistic stat = tenFromOne.copy();
+ Assertions.assertThat(stat.add(tenFromOne))
+ .isEqualTo(new MeanStatistic(2, 20))
+ .extracting(MeanStatistic::mean)
+ .isEqualTo(TEND);
+ }
+
+ @Test
+ public void testAddEmptyR() throws Throwable {
+ MeanStatistic stat = tenFromOne.copy();
+ Assertions.assertThat(stat.add(empty))
+ .isEqualTo(tenFromOne);
+ }
+
+ @Test
+ public void testAddEmptyL() throws Throwable {
+ MeanStatistic stat = empty.copy();
+ Assertions.assertThat(stat.add(tenFromOne))
+ .isEqualTo(tenFromOne);
+ }
+
+ @Test
+ public void testAddEmptyLR() throws Throwable {
+ MeanStatistic stat = empty.copy();
+ Assertions.assertThat(stat.add(empty))
+ .isEqualTo(empty);
+ }
+
+ @Test
+ public void testAddSampleToEmpty() throws Throwable {
+ MeanStatistic stat = empty.copy();
+ stat.addSample(TEN);
+ Assertions.assertThat(stat)
+ .isEqualTo(tenFromOne);
+ }
+
+ @Test
+ public void testAddZeroValueSamples() throws Throwable {
+ MeanStatistic stat = tenFromOne.copy();
+ for (int i = 0; i < 9; i++) {
+ stat.addSample(0);
+ }
+ Assertions.assertThat(stat)
+ .isEqualTo(tenFromTen);
+ }
+
+ @Test
+ public void testSetSamples() throws Throwable {
+ MeanStatistic stat = tenFromOne.copy();
+ stat.setSamples(10);
+ Assertions.assertThat(stat)
+ .isEqualTo(tenFromTen);
+ }
+
+ @Test
+ public void testSetSums() throws Throwable {
+ MeanStatistic stat = tenFromOne.copy();
+ stat.setSum(100);
+ stat.setSamples(20);
+ Assertions.assertThat(stat)
+ .isEqualTo(new MeanStatistic(20, 100))
+ .extracting(MeanStatistic::mean)
+ .isEqualTo(5.0d);
+ }
+
+ @Test
+ public void testSetNegativeSamplesMakesEmpty() throws Throwable {
+ MeanStatistic stat = tenFromOne.copy();
+ stat.setSamples(-3);
+ Assertions.assertThat(stat)
+ .isEqualTo(empty);
+ }
+
+ @Test
+ public void testRoundTrip() throws Throwable {
+ JsonSerialization serializer = serializer();
+
+ String json = serializer.toJson(tenFromTen);
+ LOG.info("serialized form\n{}", json);
+ Assertions.assertThat(json)
+ .describedAs("JSON form of %s", tenFromTen)
+ .doesNotContain("empty")
+ .doesNotContain("mean");
+
+ MeanStatistic deser = serializer.fromJson(json);
+ LOG.info("deserialized {}", deser);
+ Assertions.assertThat(deser)
+ .isEqualTo(tenFromTen);
+ }
+
+ /**
+ * negative sample counts in the json convert the stat to being empty.
+ */
+ @Test
+ public void testHandleMaliciousStat() throws Throwable {
+ String json = "{\n"
+ + " \"sum\" : 10,\n"
+ + " \"samples\" : -10\n"
+ + "}";
+ JsonSerialization serializer = serializer();
+ MeanStatistic deser = serializer.fromJson(json);
+ LOG.info("deserialized {}", deser);
+ Assertions.assertThat(deser)
+ .isEqualTo(empty);
+ }
+
+ /**
+ * Get a JSON serializer.
+ * @return a serializer.
+ */
+ public static JsonSerialization serializer() {
+ return new JsonSerialization<>(MeanStatistic.class, true, true);
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/functional/TestRemoteIterators.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/functional/TestRemoteIterators.java
new file mode 100644
index 0000000000000..f40e1303bcd69
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/functional/TestRemoteIterators.java
@@ -0,0 +1,416 @@
+/*
+ * 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.util.functional;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Iterator;
+
+import com.google.common.base.Preconditions;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.apache.hadoop.test.AbstractHadoopTestBase;
+
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.extractStatistics;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.apache.hadoop.util.functional.RemoteIterators.closingRemoteIterator;
+import static org.apache.hadoop.util.functional.RemoteIterators.filteringRemoteIterator;
+import static org.apache.hadoop.util.functional.RemoteIterators.foreach;
+import static org.apache.hadoop.util.functional.RemoteIterators.mappingRemoteIterator;
+import static org.apache.hadoop.util.functional.RemoteIterators.remoteIteratorFromArray;
+import static org.apache.hadoop.util.functional.RemoteIterators.remoteIteratorFromIterable;
+import static org.apache.hadoop.util.functional.RemoteIterators.remoteIteratorFromIterator;
+import static org.apache.hadoop.util.functional.RemoteIterators.remoteIteratorFromSingleton;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Test for {@link RemoteIterators}.
+ *
+ */
+public class TestRemoteIterators extends AbstractHadoopTestBase {
+
+ private static final Logger LOG = LoggerFactory.getLogger(
+ TestRemoteIterators.class);
+
+ private static final String[] data = {"a", "b", "c"};
+
+ /** Counter for lambda-expressions. */
+ private int counter;
+
+ @Test
+ public void testIterateArray() throws Throwable {
+ verifySize(remoteIteratorFromArray(data), data.length,
+ (s) -> LOG.info(s));
+ }
+
+ @Test
+ public void testIterateArrayMapped() throws Throwable {
+ verifySize(
+ mappingRemoteIterator(
+ remoteIteratorFromArray(data),
+ (d) -> {
+ counter += d.length();
+ return d;
+ }),
+ data.length,
+ this::log);
+ assertCounterValue(3);
+ }
+
+ public void log(Object o) {
+ LOG.info("{}", o);
+ }
+
+ /**
+ * Singleton is iterated through once.
+ * The toString() call is passed through.
+ */
+ @Test
+ public void testSingleton() throws Throwable {
+ StringBuffer result = new StringBuffer();
+ String name = "singleton";
+ RemoteIterator it = remoteIteratorFromSingleton(name);
+ assertStringValueContains(it, "SingletonIterator");
+ assertStringValueContains(it, name);
+ verifySize(
+ it,
+ 1,
+ (s) -> result.append(s));
+ assertThat(result.toString())
+ .isEqualTo(name);
+ }
+
+ @Test
+ public void testSingletonNotClosed() throws Throwable {
+ CloseCounter closeCounter = new CloseCounter();
+ RemoteIterator it = remoteIteratorFromSingleton(closeCounter);
+ verifySize(it, 1, this::log);
+ close(it);
+ closeCounter.assertCloseCount(0);
+ }
+
+ @Test
+ public void testSingletonStats() throws Throwable {
+ IOStatsInstance singleton = new IOStatsInstance();
+ RemoteIterator it
+ = remoteIteratorFromSingleton(singleton);
+ extractStatistics(it);
+ }
+
+ @Test
+ public void testMappedSingletonStats() throws Throwable {
+ IOStatsInstance singleton = new IOStatsInstance();
+ RemoteIterator it
+ = mappingRemoteIterator(remoteIteratorFromSingleton(singleton),
+ Object::toString);
+ verifySize(it, 1, this::log);
+ extractStatistics(it);
+ }
+
+ @Test
+ public void testIteratorPassthrough() throws Throwable {
+ CountdownRemoteIterator it = new CountdownRemoteIterator(0);
+ verifySize(it, 0, this::log);
+ extractStatistics(it);
+ it.close();
+ it.assertCloseCount(1);
+ }
+
+ @Test
+ public void testMapping() throws Throwable {
+ CountdownRemoteIterator countdown = new CountdownRemoteIterator(100);
+ RemoteIterator it = mappingRemoteIterator(
+ countdown,
+ i -> i);
+ verifySize(it, 100, c -> counter++);
+ assertCounterValue(100);
+ extractStatistics(it);
+ assertStringValueContains(it, "CountdownRemoteIterator");
+ close(it);
+ countdown.assertCloseCount(1);
+ }
+
+ @Test
+ public void testFiltering() throws Throwable {
+ CountdownRemoteIterator countdown = new CountdownRemoteIterator(100);
+ // only even numbers are passed through
+ RemoteIterator it = filteringRemoteIterator(
+ countdown,
+ i -> (i % 2) == 0);
+ verifySize(it, 50, c -> counter++);
+ assertCounterValue(50);
+ extractStatistics(it);
+ close(it);
+ countdown.assertCloseCount(1);
+ }
+
+ @Test
+ public void testFilterNoneAccepted() throws Throwable {
+ // nothing gets through
+ RemoteIterator it = filteringRemoteIterator(
+ new CountdownRemoteIterator(100),
+ i -> false);
+ verifySize(it, 0, c -> counter++);
+ assertCounterValue(0);
+ extractStatistics(it);
+ }
+
+ @Test
+ public void testFilterAllAccepted() throws Throwable {
+ // nothing gets through
+ RemoteIterator it = filteringRemoteIterator(
+ new CountdownRemoteIterator(100),
+ i -> true);
+ verifySize(it, 100, c -> counter++);
+ assertStringValueContains(it, "CountdownRemoteIterator");
+ }
+
+ @Test
+ public void testIteratorSupport() throws Throwable {
+ CountdownIterator countdownIterator = new CountdownIterator(100);
+ RemoteIterator it = remoteIteratorFromIterator(
+ countdownIterator);
+ verifySize(it, 100, c -> counter++);
+ assertStringValueContains(it, "CountdownIterator");
+ extractStatistics(it);
+ close(it);
+ countdownIterator.assertCloseCount(1);
+ }
+
+ @Test
+ public void testIterableSupport() throws Throwable {
+ CountdownIterable countdown = new CountdownIterable(100);
+ RemoteIterator it = remoteIteratorFromIterable(
+ countdown);
+ verifySize(it, 100, c -> counter++);
+ assertStringValueContains(it, "CountdownIterator");
+ extractStatistics(it);
+ // close the iterator
+ close(it);
+ countdown.assertCloseCount(0);
+ // and a new iterator can be crated
+ verifySize(remoteIteratorFromIterable(countdown),
+ 100, c -> counter++);
+
+ }
+
+ /**
+ * If a RemoteIterator is constructed from an iterable
+ * and that is to be closed, we add it.
+ * @throws Throwable
+ */
+ @Test
+ public void testIterableClose() throws Throwable {
+ CountdownIterable countdown = new CountdownIterable(100);
+ RemoteIterator it = closingRemoteIterator(
+ remoteIteratorFromIterable(countdown),
+ countdown);
+ verifySize(it, 100, c -> counter++);
+ assertStringValueContains(it, "CountdownIterator");
+ extractStatistics(it);
+ // close the iterator
+ close(it);
+ countdown.assertCloseCount(1);
+ // and a new iterator cannot be created
+ intercept(IllegalStateException.class, () ->
+ remoteIteratorFromIterable(countdown));
+ }
+
+ /**
+ * assert that the string value of an object contains the
+ * expected text.
+ * @param o object
+ * @param expected expected text
+ */
+ protected void assertStringValueContains(
+ final Object o,
+ final String expected) {
+ assertThat(o.toString())
+ .describedAs("Iterator string value")
+ .contains(expected);
+ }
+
+ /**
+ * Assert that the counter field is at a specific value.
+ * @param expected counter
+ */
+ protected void assertCounterValue(final int expected) {
+ assertThat(counter)
+ .describedAs("Counter value")
+ .isEqualTo(expected);
+ }
+
+ /**
+ * Verify that the iteration completes with a given size.
+ * @param it iterator
+ * @param type.
+ * @param length expected size
+ * @param consumer consumer
+ */
+ protected void verifySize(final RemoteIterator