Skip to content

Commit 9b2956e

Browse files
MAPREDUCE-7315. LocatedFileStatusFetcher to collect/publish IOStatistics. (#2579)
Part of the HADOOP-16830 IOStatistics API feature. If the source FileSystem's listing RemoteIterators implement IOStatisticsSource, these are collected and served through the IOStatisticsSource API. If they are not: getIOStatistics() returns null. Only the listing statistics are collected; FileSystem.globStatus() doesn't provide any, so IO use there is not included in the aggregate results. Contributed by Steve Loughran.
1 parent 99d08a1 commit 9b2956e

File tree

1 file changed

+58
-2
lines changed
  • hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred

1 file changed

+58
-2
lines changed

hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LocatedFileStatusFetcher.java

Lines changed: 58 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@
2121
import java.io.IOException;
2222
import java.util.LinkedList;
2323
import java.util.List;
24+
import java.util.StringJoiner;
2425
import java.util.concurrent.BlockingQueue;
2526
import java.util.concurrent.Callable;
2627
import java.util.concurrent.ExecutorService;
@@ -37,6 +38,9 @@
3738
import org.apache.hadoop.fs.Path;
3839
import org.apache.hadoop.fs.PathFilter;
3940
import org.apache.hadoop.fs.RemoteIterator;
41+
import org.apache.hadoop.fs.statistics.IOStatistics;
42+
import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
43+
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
4044
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
4145

4246
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
@@ -52,6 +56,9 @@
5256

5357
import org.apache.hadoop.util.concurrent.HadoopExecutors;
5458

59+
import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics;
60+
import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics;
61+
5562
/**
5663
* Utility class to fetch block locations for specified Input paths using a
5764
* configured number of threads.
@@ -60,7 +67,7 @@
6067
* configuration.
6168
*/
6269
@Private
63-
public class LocatedFileStatusFetcher {
70+
public class LocatedFileStatusFetcher implements IOStatisticsSource {
6471

6572
public static final Logger LOG =
6673
LoggerFactory.getLogger(LocatedFileStatusFetcher.class.getName());
@@ -87,6 +94,12 @@ public class LocatedFileStatusFetcher {
8794

8895
private volatile Throwable unknownError;
8996

97+
/**
98+
* Demand created IO Statistics: only if the filesystem
99+
* returns statistics does this fetch collect them.
100+
*/
101+
private IOStatisticsSnapshot iostats;
102+
90103
/**
91104
* Instantiate.
92105
* The newApi switch is only used to configure what exception is raised
@@ -226,7 +239,46 @@ private void decrementRunningAndCheckCompletion() {
226239
lock.unlock();
227240
}
228241
}
229-
242+
243+
/**
244+
* Return any IOStatistics collected during listing.
245+
* @return IO stats accrued.
246+
*/
247+
@Override
248+
public synchronized IOStatistics getIOStatistics() {
249+
return iostats;
250+
}
251+
252+
/**
253+
* Add the statistics of an individual thread's scan.
254+
* @param stats possibly null statistics.
255+
*/
256+
private void addResultStatistics(IOStatistics stats) {
257+
if (stats != null) {
258+
// demand creation of IO statistics.
259+
synchronized (this) {
260+
LOG.debug("Adding IOStatistics: {}", stats);
261+
if (iostats == null) {
262+
// demand create the statistics
263+
iostats = snapshotIOStatistics(stats);
264+
} else {
265+
iostats.aggregate(stats);
266+
}
267+
}
268+
}
269+
}
270+
271+
@Override
272+
public String toString() {
273+
final IOStatistics ioStatistics = getIOStatistics();
274+
StringJoiner stringJoiner = new StringJoiner(", ",
275+
LocatedFileStatusFetcher.class.getSimpleName() + "[", "]");
276+
if (ioStatistics != null) {
277+
stringJoiner.add("IOStatistics=" + ioStatistics);
278+
}
279+
return stringJoiner.toString();
280+
}
281+
230282
/**
231283
* Retrieves block locations for the given @link {@link FileStatus}, and adds
232284
* additional paths to the process queue if required.
@@ -266,6 +318,8 @@ public Result call() throws Exception {
266318
}
267319
}
268320
}
321+
// aggregate any stats
322+
result.stats = retrieveIOStatistics(iter);
269323
} else {
270324
result.locatedFileStatuses.add(fileStatus);
271325
}
@@ -276,6 +330,7 @@ private static class Result {
276330
private List<FileStatus> locatedFileStatuses = new LinkedList<>();
277331
private List<FileStatus> dirsNeedingRecursiveCalls = new LinkedList<>();
278332
private FileSystem fs;
333+
private IOStatistics stats;
279334
}
280335
}
281336

@@ -290,6 +345,7 @@ private class ProcessInputDirCallback implements
290345
@Override
291346
public void onSuccess(ProcessInputDirCallable.Result result) {
292347
try {
348+
addResultStatistics(result.stats);
293349
if (!result.locatedFileStatuses.isEmpty()) {
294350
resultQueue.add(result.locatedFileStatuses);
295351
}

0 commit comments

Comments
 (0)