|
@@ -18,29 +18,42 @@
|
|
|
|
|
|
package org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages;
|
|
package org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages;
|
|
|
|
|
|
|
|
+import java.io.File;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
|
|
+import java.util.Map;
|
|
|
|
+import java.util.concurrent.ConcurrentHashMap;
|
|
|
|
+import java.util.concurrent.atomic.AtomicLong;
|
|
|
|
+import java.util.stream.Collectors;
|
|
|
|
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.LoggerFactory;
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
|
|
+import org.apache.hadoop.classification.VisibleForTesting;
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.PathIOException;
|
|
import org.apache.hadoop.fs.PathIOException;
|
|
import org.apache.hadoop.fs.RemoteIterator;
|
|
import org.apache.hadoop.fs.RemoteIterator;
|
|
import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
|
|
import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
|
|
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
|
|
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
|
|
|
|
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
|
|
|
|
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DirEntry;
|
|
import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest;
|
|
import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest;
|
|
|
|
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.EntryFileIO;
|
|
|
|
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.LoadedManifestData;
|
|
import org.apache.hadoop.util.functional.TaskPool;
|
|
import org.apache.hadoop.util.functional.TaskPool;
|
|
|
|
|
|
import static org.apache.commons.io.FileUtils.byteCountToDisplaySize;
|
|
import static org.apache.commons.io.FileUtils.byteCountToDisplaySize;
|
|
import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics;
|
|
import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics;
|
|
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation;
|
|
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation;
|
|
|
|
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_TASK_DIRECTORY_COUNT_MEAN;
|
|
|
|
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_TASK_FILE_COUNT_MEAN;
|
|
import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_TASK_MANIFEST_FILE_SIZE;
|
|
import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_TASK_MANIFEST_FILE_SIZE;
|
|
import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_LOAD_ALL_MANIFESTS;
|
|
import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_LOAD_ALL_MANIFESTS;
|
|
import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_LOAD_MANIFESTS;
|
|
import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_LOAD_MANIFESTS;
|
|
import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.maybeAddIOStatistics;
|
|
import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.maybeAddIOStatistics;
|
|
|
|
+import static org.apache.hadoop.util.functional.RemoteIterators.haltableRemoteIterator;
|
|
|
|
|
|
/**
|
|
/**
|
|
* Stage to load all the task manifests in the job attempt directory.
|
|
* Stage to load all the task manifests in the job attempt directory.
|
|
@@ -52,7 +65,7 @@ import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.Man
|
|
*/
|
|
*/
|
|
public class LoadManifestsStage extends
|
|
public class LoadManifestsStage extends
|
|
AbstractJobOrTaskStage<
|
|
AbstractJobOrTaskStage<
|
|
- Boolean,
|
|
|
|
|
|
+ LoadManifestsStage.Arguments,
|
|
LoadManifestsStage.Result> {
|
|
LoadManifestsStage.Result> {
|
|
|
|
|
|
private static final Logger LOG = LoggerFactory.getLogger(
|
|
private static final Logger LOG = LoggerFactory.getLogger(
|
|
@@ -64,14 +77,14 @@ public class LoadManifestsStage extends
|
|
private final SummaryInfo summaryInfo = new SummaryInfo();
|
|
private final SummaryInfo summaryInfo = new SummaryInfo();
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Should manifests be pruned of IOStatistics?
|
|
|
|
|
|
+ * Map of directories from manifests, coalesced to reduce duplication.
|
|
*/
|
|
*/
|
|
- private boolean pruneManifests;
|
|
|
|
|
|
+ private final Map<String, DirEntry> directories = new ConcurrentHashMap<>();
|
|
|
|
|
|
/**
|
|
/**
|
|
- * List of loaded manifests.
|
|
|
|
|
|
+ * Writer of entries.
|
|
*/
|
|
*/
|
|
- private final List<TaskManifest> manifests = new ArrayList<>();
|
|
|
|
|
|
+ private EntryFileIO.EntryWriter entryWriter;
|
|
|
|
|
|
public LoadManifestsStage(final StageConfig stageConfig) {
|
|
public LoadManifestsStage(final StageConfig stageConfig) {
|
|
super(false, stageConfig, OP_STAGE_JOB_LOAD_MANIFESTS, true);
|
|
super(false, stageConfig, OP_STAGE_JOB_LOAD_MANIFESTS, true);
|
|
@@ -79,43 +92,83 @@ public class LoadManifestsStage extends
|
|
|
|
|
|
/**
|
|
/**
|
|
* Load the manifests.
|
|
* Load the manifests.
|
|
- * @param prune should manifests be pruned of IOStatistics?
|
|
|
|
|
|
+ * @param arguments stage arguments
|
|
* @return the summary and a list of manifests.
|
|
* @return the summary and a list of manifests.
|
|
* @throws IOException IO failure.
|
|
* @throws IOException IO failure.
|
|
*/
|
|
*/
|
|
@Override
|
|
@Override
|
|
protected LoadManifestsStage.Result executeStage(
|
|
protected LoadManifestsStage.Result executeStage(
|
|
- final Boolean prune) throws IOException {
|
|
|
|
|
|
+ final LoadManifestsStage.Arguments arguments) throws IOException {
|
|
|
|
+
|
|
|
|
+ EntryFileIO entryFileIO = new EntryFileIO(getStageConfig().getConf());
|
|
|
|
|
|
final Path manifestDir = getTaskManifestDir();
|
|
final Path manifestDir = getTaskManifestDir();
|
|
LOG.info("{}: Executing Manifest Job Commit with manifests in {}",
|
|
LOG.info("{}: Executing Manifest Job Commit with manifests in {}",
|
|
getName(),
|
|
getName(),
|
|
manifestDir);
|
|
manifestDir);
|
|
- pruneManifests = prune;
|
|
|
|
- // build a list of all task manifests successfully committed
|
|
|
|
- //
|
|
|
|
- msync(manifestDir);
|
|
|
|
- final RemoteIterator<FileStatus> manifestFiles = listManifests();
|
|
|
|
-
|
|
|
|
- final List<TaskManifest> manifestList = loadAllManifests(manifestFiles);
|
|
|
|
- LOG.info("{}: Summary of {} manifests loaded in {}: {}",
|
|
|
|
- getName(),
|
|
|
|
- manifestList.size(),
|
|
|
|
- manifestDir,
|
|
|
|
- summaryInfo);
|
|
|
|
|
|
|
|
- // collect any stats
|
|
|
|
- maybeAddIOStatistics(getIOStatistics(), manifestFiles);
|
|
|
|
- return new LoadManifestsStage.Result(summaryInfo, manifestList);
|
|
|
|
|
|
+ final Path entrySequenceData = arguments.getEntrySequenceData();
|
|
|
|
+
|
|
|
|
+ // the entry writer for queuing data.
|
|
|
|
+ entryWriter = entryFileIO.launchEntryWriter(
|
|
|
|
+ entryFileIO.createWriter(entrySequenceData),
|
|
|
|
+ arguments.queueCapacity);
|
|
|
|
+
|
|
|
|
+ try {
|
|
|
|
+
|
|
|
|
+ // sync fs before the list
|
|
|
|
+ msync(manifestDir);
|
|
|
|
+
|
|
|
|
+ // build a list of all task manifests successfully committed,
|
|
|
|
+ // which will break out if the writing is stopped (due to any failure)
|
|
|
|
+ final RemoteIterator<FileStatus> manifestFiles =
|
|
|
|
+ haltableRemoteIterator(listManifests(),
|
|
|
|
+ () -> entryWriter.isActive());
|
|
|
|
+
|
|
|
|
+ processAllManifests(manifestFiles);
|
|
|
|
+ maybeAddIOStatistics(getIOStatistics(), manifestFiles);
|
|
|
|
+
|
|
|
|
+ LOG.info("{}: Summary of {} manifests loaded in {}: {}",
|
|
|
|
+ getName(),
|
|
|
|
+ summaryInfo.manifestCount,
|
|
|
|
+ manifestDir,
|
|
|
|
+ summaryInfo);
|
|
|
|
+
|
|
|
|
+ // close cleanly
|
|
|
|
+ entryWriter.close();
|
|
|
|
+
|
|
|
|
+ // if anything failed, raise it.
|
|
|
|
+ entryWriter.maybeRaiseWriteException();
|
|
|
|
+
|
|
|
|
+ // collect any stats
|
|
|
|
+ } catch (EntryWriteException e) {
|
|
|
|
+ // something went wrong while writing.
|
|
|
|
+ // raise anything on the write thread,
|
|
|
|
+ entryWriter.maybeRaiseWriteException();
|
|
|
|
+
|
|
|
|
+ // falling back to that from the worker thread
|
|
|
|
+ throw e;
|
|
|
|
+ } finally {
|
|
|
|
+ // close which is a no-op if the clean close was invoked;
|
|
|
|
+ // it is not a no-op if something went wrong with reading/parsing/processing
|
|
|
|
+ // the manifests.
|
|
|
|
+ entryWriter.close();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ final LoadedManifestData loadedManifestData = new LoadedManifestData(
|
|
|
|
+ new ArrayList<>(directories.values()), // new array to free up the map
|
|
|
|
+ entrySequenceData,
|
|
|
|
+ entryWriter.getCount());
|
|
|
|
+
|
|
|
|
+ return new LoadManifestsStage.Result(summaryInfo, loadedManifestData);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Load all the manifests.
|
|
|
|
|
|
+ * Load and process all the manifests.
|
|
* @param manifestFiles list of manifest files.
|
|
* @param manifestFiles list of manifest files.
|
|
- * @return the loaded manifests.
|
|
|
|
- * @throws IOException IO Failure.
|
|
|
|
|
|
+ * @throws IOException failure to load/parse/queue
|
|
*/
|
|
*/
|
|
- private List<TaskManifest> loadAllManifests(
|
|
|
|
|
|
+ private void processAllManifests(
|
|
final RemoteIterator<FileStatus> manifestFiles) throws IOException {
|
|
final RemoteIterator<FileStatus> manifestFiles) throws IOException {
|
|
|
|
|
|
trackDurationOfInvocation(getIOStatistics(), OP_LOAD_ALL_MANIFESTS, () ->
|
|
trackDurationOfInvocation(getIOStatistics(), OP_LOAD_ALL_MANIFESTS, () ->
|
|
@@ -123,33 +176,73 @@ public class LoadManifestsStage extends
|
|
.executeWith(getIOProcessors())
|
|
.executeWith(getIOProcessors())
|
|
.stopOnFailure()
|
|
.stopOnFailure()
|
|
.run(this::processOneManifest));
|
|
.run(this::processOneManifest));
|
|
- return manifests;
|
|
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
* Method invoked to process one manifest.
|
|
* Method invoked to process one manifest.
|
|
* @param status file to process.
|
|
* @param status file to process.
|
|
- * @throws IOException failure to load/parse
|
|
|
|
|
|
+ * @throws IOException failure to load/parse/queue
|
|
*/
|
|
*/
|
|
private void processOneManifest(FileStatus status)
|
|
private void processOneManifest(FileStatus status)
|
|
throws IOException {
|
|
throws IOException {
|
|
updateAuditContext(OP_LOAD_ALL_MANIFESTS);
|
|
updateAuditContext(OP_LOAD_ALL_MANIFESTS);
|
|
|
|
|
|
- TaskManifest m = fetchTaskManifest(status);
|
|
|
|
|
|
+ TaskManifest manifest = fetchTaskManifest(status);
|
|
progress();
|
|
progress();
|
|
|
|
|
|
- // update the manifest list in a synchronized block.
|
|
|
|
-
|
|
|
|
- synchronized (manifests) {
|
|
|
|
- manifests.add(m);
|
|
|
|
- // and the summary info in the same block, to
|
|
|
|
- // eliminate the need to acquire a second lock.
|
|
|
|
- summaryInfo.add(m);
|
|
|
|
|
|
+ // update the directories
|
|
|
|
+ final int created = coalesceDirectories(manifest);
|
|
|
|
+ final String attemptID = manifest.getTaskAttemptID();
|
|
|
|
+ LOG.debug("{}: task attempt {} added {} directories",
|
|
|
|
+ getName(), attemptID, created);
|
|
|
|
+
|
|
|
|
+ // add to the summary.
|
|
|
|
+ summaryInfo.add(manifest);
|
|
|
|
+
|
|
|
|
+ // clear the manifest extra data so if
|
|
|
|
+ // blocked waiting for queue capacity,
|
|
|
|
+ // memory use is reduced.
|
|
|
|
+ manifest.setIOStatistics(null);
|
|
|
|
+ manifest.getExtraData().clear();
|
|
|
|
+
|
|
|
|
+ // queue those files.
|
|
|
|
+ final boolean enqueued = entryWriter.enqueue(manifest.getFilesToCommit());
|
|
|
|
+ if (!enqueued) {
|
|
|
|
+ LOG.warn("{}: Failed to write manifest for task {}",
|
|
|
|
+ getName(), attemptID);
|
|
|
|
+ throw new EntryWriteException(attemptID);
|
|
}
|
|
}
|
|
- if (pruneManifests) {
|
|
|
|
- m.setIOStatistics(null);
|
|
|
|
- m.getExtraData().clear();
|
|
|
|
|
|
+
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Coalesce all directories and clear the entry in the manifest.
|
|
|
|
+ * There's only ever one writer at a time, which it is hoped reduces
|
|
|
|
+ * contention. before the lock is acquired: if there are no new directories,
|
|
|
|
+ * the write lock is never needed.
|
|
|
|
+ * @param manifest manifest to process
|
|
|
|
+ * @return the number of directories created;
|
|
|
|
+ */
|
|
|
|
+ @VisibleForTesting
|
|
|
|
+ int coalesceDirectories(final TaskManifest manifest) {
|
|
|
|
+
|
|
|
|
+ // build a list of dirs to create.
|
|
|
|
+ // this scans the map
|
|
|
|
+ final List<DirEntry> toCreate = manifest.getDestDirectories().stream()
|
|
|
|
+ .filter(e -> !directories.containsKey(e))
|
|
|
|
+ .collect(Collectors.toList());
|
|
|
|
+ if (!toCreate.isEmpty()) {
|
|
|
|
+ // need to add more directories;
|
|
|
|
+ // still a possibility that they may be created between the
|
|
|
|
+ // filtering and this thread having the write lock.
|
|
|
|
+
|
|
|
|
+ synchronized (directories) {
|
|
|
|
+ toCreate.forEach(entry -> {
|
|
|
|
+ directories.putIfAbsent(entry.getDir(), entry);
|
|
|
|
+ });
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
+ return toCreate.size();
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -173,55 +266,121 @@ public class LoadManifestsStage extends
|
|
final long size = manifest.getTotalFileSize();
|
|
final long size = manifest.getTotalFileSize();
|
|
LOG.info("{}: Task Attempt {} file {}: File count: {}; data size={}",
|
|
LOG.info("{}: Task Attempt {} file {}: File count: {}; data size={}",
|
|
getName(), id, status.getPath(), filecount, size);
|
|
getName(), id, status.getPath(), filecount, size);
|
|
- // record file size for tracking of memory consumption.
|
|
|
|
- getIOStatistics().addMeanStatisticSample(COMMITTER_TASK_MANIFEST_FILE_SIZE,
|
|
|
|
- status.getLen());
|
|
|
|
|
|
+
|
|
|
|
+ // record file size for tracking of memory consumption, work etc.
|
|
|
|
+ final IOStatisticsStore iostats = getIOStatistics();
|
|
|
|
+ iostats.addSample(COMMITTER_TASK_MANIFEST_FILE_SIZE, status.getLen());
|
|
|
|
+ iostats.addSample(COMMITTER_TASK_FILE_COUNT_MEAN, filecount);
|
|
|
|
+ iostats.addSample(COMMITTER_TASK_DIRECTORY_COUNT_MEAN,
|
|
|
|
+ manifest.getDestDirectories().size());
|
|
return manifest;
|
|
return manifest;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Stage arguments.
|
|
|
|
+ */
|
|
|
|
+ public static final class Arguments {
|
|
|
|
+ /**
|
|
|
|
+ * File where the listing has been saved.
|
|
|
|
+ */
|
|
|
|
+ private final File entrySequenceFile;
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Capacity for queue between manifest loader and the writers.
|
|
|
|
+ */
|
|
|
|
+ private final int queueCapacity;
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Arguments.
|
|
|
|
+ * @param entrySequenceFile path to local file to create for storing entries
|
|
|
|
+ * @param queueCapacity capacity of the queue
|
|
|
|
+ */
|
|
|
|
+ public Arguments(
|
|
|
|
+ final File entrySequenceFile,
|
|
|
|
+ final int queueCapacity) {
|
|
|
|
+ this.entrySequenceFile = entrySequenceFile;
|
|
|
|
+ this.queueCapacity = queueCapacity;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private Path getEntrySequenceData() {
|
|
|
|
+ return new Path(entrySequenceFile.toURI());
|
|
|
|
+
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Result of the stage.
|
|
* Result of the stage.
|
|
*/
|
|
*/
|
|
public static final class Result {
|
|
public static final class Result {
|
|
private final SummaryInfo summary;
|
|
private final SummaryInfo summary;
|
|
|
|
|
|
- private final List<TaskManifest> manifests;
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Output of this stage to pass on to the subsequence stages.
|
|
|
|
+ */
|
|
|
|
+ private final LoadedManifestData loadedManifestData;
|
|
|
|
|
|
- public Result(SummaryInfo summary,
|
|
|
|
- List<TaskManifest> manifests) {
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Result.
|
|
|
|
+ * @param summary summary of jobs
|
|
|
|
+ * @param loadedManifestData all loaded manifest data
|
|
|
|
+ */
|
|
|
|
+ public Result(
|
|
|
|
+ final SummaryInfo summary,
|
|
|
|
+ final LoadedManifestData loadedManifestData) {
|
|
this.summary = summary;
|
|
this.summary = summary;
|
|
- this.manifests = manifests;
|
|
|
|
|
|
+ this.loadedManifestData = loadedManifestData;
|
|
}
|
|
}
|
|
|
|
|
|
public SummaryInfo getSummary() {
|
|
public SummaryInfo getSummary() {
|
|
return summary;
|
|
return summary;
|
|
}
|
|
}
|
|
|
|
|
|
- public List<TaskManifest> getManifests() {
|
|
|
|
- return manifests;
|
|
|
|
|
|
+ public LoadedManifestData getLoadedManifestData() {
|
|
|
|
+ return loadedManifestData;
|
|
}
|
|
}
|
|
-
|
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * IOE to raise on queueing failure.
|
|
|
|
+ */
|
|
|
|
+ public static final class EntryWriteException extends IOException {
|
|
|
|
+
|
|
|
|
+ private EntryWriteException(String taskId) {
|
|
|
|
+ super("Failed to write manifest data for task "
|
|
|
|
+ + taskId + "to local file");
|
|
|
|
+ }
|
|
|
|
+ }
|
|
/**
|
|
/**
|
|
* Summary information.
|
|
* Summary information.
|
|
|
|
+ * Implementation note: atomic counters are used here to keep spotbugs quiet,
|
|
|
|
+ * not because of any concurrency risks.
|
|
*/
|
|
*/
|
|
public static final class SummaryInfo implements IOStatisticsSource {
|
|
public static final class SummaryInfo implements IOStatisticsSource {
|
|
|
|
|
|
/**
|
|
/**
|
|
* Aggregate IOStatistics.
|
|
* Aggregate IOStatistics.
|
|
*/
|
|
*/
|
|
- private IOStatisticsSnapshot iostatistics = snapshotIOStatistics();
|
|
|
|
|
|
+ private final IOStatisticsSnapshot iostatistics = snapshotIOStatistics();
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Task IDs.
|
|
|
|
+ */
|
|
|
|
+ private final List<String> taskIDs = new ArrayList<>();
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Task IDs.
|
|
|
|
+ */
|
|
|
|
+ private final List<String> taskAttemptIDs = new ArrayList<>();
|
|
|
|
|
|
/**
|
|
/**
|
|
* How many manifests were loaded.
|
|
* How many manifests were loaded.
|
|
*/
|
|
*/
|
|
- private long manifestCount;
|
|
|
|
|
|
+ private AtomicLong manifestCount = new AtomicLong();
|
|
|
|
|
|
/**
|
|
/**
|
|
* Total number of files to rename.
|
|
* Total number of files to rename.
|
|
*/
|
|
*/
|
|
- private long fileCount;
|
|
|
|
|
|
+ private AtomicLong fileCount = new AtomicLong();
|
|
|
|
|
|
/**
|
|
/**
|
|
* Total number of directories which may need
|
|
* Total number of directories which may need
|
|
@@ -229,12 +388,12 @@ public class LoadManifestsStage extends
|
|
* As there is no dedup, this is likely to be
|
|
* As there is no dedup, this is likely to be
|
|
* a (major) overestimate.
|
|
* a (major) overestimate.
|
|
*/
|
|
*/
|
|
- private long directoryCount;
|
|
|
|
|
|
+ private AtomicLong directoryCount = new AtomicLong();
|
|
|
|
|
|
/**
|
|
/**
|
|
* Total amount of data to be committed.
|
|
* Total amount of data to be committed.
|
|
*/
|
|
*/
|
|
- private long totalFileSize;
|
|
|
|
|
|
+ private AtomicLong totalFileSize = new AtomicLong();
|
|
|
|
|
|
/**
|
|
/**
|
|
* Get the IOStatistics.
|
|
* Get the IOStatistics.
|
|
@@ -246,31 +405,41 @@ public class LoadManifestsStage extends
|
|
}
|
|
}
|
|
|
|
|
|
public long getFileCount() {
|
|
public long getFileCount() {
|
|
- return fileCount;
|
|
|
|
|
|
+ return fileCount.get();
|
|
}
|
|
}
|
|
|
|
|
|
public long getDirectoryCount() {
|
|
public long getDirectoryCount() {
|
|
- return directoryCount;
|
|
|
|
|
|
+ return directoryCount.get();
|
|
}
|
|
}
|
|
|
|
|
|
public long getTotalFileSize() {
|
|
public long getTotalFileSize() {
|
|
- return totalFileSize;
|
|
|
|
|
|
+ return totalFileSize.get();
|
|
}
|
|
}
|
|
|
|
|
|
public long getManifestCount() {
|
|
public long getManifestCount() {
|
|
- return manifestCount;
|
|
|
|
|
|
+ return manifestCount.get();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public List<String> getTaskIDs() {
|
|
|
|
+ return taskIDs;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public List<String> getTaskAttemptIDs() {
|
|
|
|
+ return taskAttemptIDs;
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Add all statistics.
|
|
|
|
|
|
+ * Add all statistics; synchronized.
|
|
* @param manifest manifest to add.
|
|
* @param manifest manifest to add.
|
|
*/
|
|
*/
|
|
- public void add(TaskManifest manifest) {
|
|
|
|
- manifestCount++;
|
|
|
|
|
|
+ public synchronized void add(TaskManifest manifest) {
|
|
|
|
+ manifestCount.incrementAndGet();
|
|
iostatistics.aggregate(manifest.getIOStatistics());
|
|
iostatistics.aggregate(manifest.getIOStatistics());
|
|
- fileCount += manifest.getFilesToCommit().size();
|
|
|
|
- directoryCount += manifest.getDestDirectories().size();
|
|
|
|
- totalFileSize += manifest.getTotalFileSize();
|
|
|
|
|
|
+ fileCount.addAndGet(manifest.getFilesToCommit().size());
|
|
|
|
+ directoryCount.addAndGet(manifest.getDestDirectories().size());
|
|
|
|
+ totalFileSize.addAndGet(manifest.getTotalFileSize());
|
|
|
|
+ taskIDs.add(manifest.getTaskID());
|
|
|
|
+ taskAttemptIDs.add(manifest.getTaskAttemptID());
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -281,11 +450,11 @@ public class LoadManifestsStage extends
|
|
public String toString() {
|
|
public String toString() {
|
|
final StringBuilder sb = new StringBuilder(
|
|
final StringBuilder sb = new StringBuilder(
|
|
"SummaryInfo{");
|
|
"SummaryInfo{");
|
|
- sb.append("manifestCount=").append(manifestCount);
|
|
|
|
- sb.append(", fileCount=").append(fileCount);
|
|
|
|
- sb.append(", directoryCount=").append(directoryCount);
|
|
|
|
|
|
+ sb.append("manifestCount=").append(getManifestCount());
|
|
|
|
+ sb.append(", fileCount=").append(getFileCount());
|
|
|
|
+ sb.append(", directoryCount=").append(getDirectoryCount());
|
|
sb.append(", totalFileSize=").append(
|
|
sb.append(", totalFileSize=").append(
|
|
- byteCountToDisplaySize(totalFileSize));
|
|
|
|
|
|
+ byteCountToDisplaySize(getTotalFileSize()));
|
|
sb.append('}');
|
|
sb.append('}');
|
|
return sb.toString();
|
|
return sb.toString();
|
|
}
|
|
}
|