From 17db68e04dacee93206aa1d977a02f2060bfe936 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 29 Mar 2023 15:24:47 +0100 Subject: [PATCH 01/18] MAPREDUCE-7435. Manifest Committer OOM on abfs * Add heap information to gauges in _SUCCESS * which includes pulling up part of impl.IOStatisticsStore into a public IOStatisticsSetters interface with the put{Counter, Gauge, etc} methods only. * TestLoadManifests scaled up with #of tasks and #of files in each task to generate more load. * code to build up dir map during load phase; not wired up Summary: abfs uses a lot more heap during the load phase than file; possibly due to buffering, but doing a pipeline for processing the results isn't sustainable. Either two phase: * phase 1, build up the dir list, discard manifests after each load * phase 2, load manifests and rename incrementally Or: unified with some complicated directory creation process to ensure that each task's dirs exist before its rename begins. Change-Id: I8595c083435e3d4df27343599687677abfc1c013 --- .../fs/statistics/IOStatisticsSetters.java | 75 ++++++++++++ .../fs/statistics/IOStatisticsSnapshot.java | 30 ++++- .../fs/statistics/impl/IOStatisticsStore.java | 48 +------- .../committer/manifest/ManifestCommitter.java | 8 +- .../manifest/files/DiagnosticKeys.java | 3 + .../impl/ManifestCommitterSupport.java | 24 ++++ .../manifest/stages/CommitJobStage.java | 14 ++- .../manifest/stages/LoadManifestsStage.java | 29 ++++- .../lib/output/TestFileOutputCommitter.java | 111 ++++++++++-------- .../ManifestCommitterTestSupport.java | 2 +- .../manifest/TestLoadManifestsStage.java | 65 +++++++++- .../azurebfs/commit/AbfsCommitTestHelper.java | 3 + .../commit/ITestAbfsLoadManifestsStage.java | 6 + 13 files changed, 308 insertions(+), 110 deletions(-) create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSetters.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSetters.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSetters.java new file mode 100644 index 0000000000000..8ad137541c146 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSetters.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; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Setter for IOStatistics entries. + * These operations have been in the read/write API + * {@code IOStatisticsStore} since IOStatistics + * was added; extracting into its own interface allows for + * {@link IOStatisticsSnapshot} to also support it. + * These are the simple setters, they don't provide for increments, + * decrements, calculation of min/max/mean etc. + * @since The interface and IOStatisticsSnapshot support came after Hadoop 3.3.5 + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public interface IOStatisticsSetters { + + /** + * 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. + * + * @param key statistics key + * @param value value to set + */ + void setGauge(String key, long value); + + /** + * Set a maximum. + * @param key statistics key + * @param value value to set + */ + void setMaximum(String key, long value); + + /** + * Set a minimum. + * @param key statistics key + * @param value value to set + */ + void setMinimum(String key, long value); + + /** + * Set a mean statistic to a given value. + * @param key statistic key + * @param value new value. + */ + void setMeanStatistic(String key, MeanStatistic value); +} 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 index 88606eb4b3055..e855a3f611e4d 100644 --- 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 @@ -62,7 +62,8 @@ @InterfaceAudience.Public @InterfaceStability.Evolving public final class IOStatisticsSnapshot - implements IOStatistics, Serializable, IOStatisticsAggregator { + implements IOStatistics, Serializable, IOStatisticsAggregator, + IOStatisticsSetters{ private static final long serialVersionUID = -1762522703841538084L; @@ -222,6 +223,33 @@ public synchronized Map meanStatistics() { return meanStatistics; } + @Override + public synchronized void setCounter(final String key, final long value) { + counters().put(key, value); + } + + @Override + public synchronized void setGauge(final String key, final long value) { + gauges().put(key, value); + + } + + @Override + public synchronized void setMaximum(final String key, final long value) { + maximums().put(key, value); + + } + + @Override + public synchronized void setMinimum(final String key, final long value) { + minimums().put(key, value); + } + + @Override + public void setMeanStatistic(final String key, final MeanStatistic value) { + + } + @Override public String toString() { return ioStatisticsToString(this); 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 index c083ad8c3c2ed..fed7c69ccd85d 100644 --- 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 @@ -24,6 +24,7 @@ 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.IOStatisticsSetters; import org.apache.hadoop.fs.statistics.MeanStatistic; /** @@ -31,6 +32,7 @@ * use in classes which track statistics for reporting. */ public interface IOStatisticsStore extends IOStatistics, + IOStatisticsSetters, IOStatisticsAggregator, DurationTrackerFactory { @@ -56,24 +58,6 @@ default long incrementCounter(String key) { */ 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. *

@@ -85,14 +69,6 @@ default long incrementCounter(String key) { */ 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. *

@@ -104,16 +80,6 @@ default long incrementCounter(String key) { */ 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. *

@@ -147,16 +113,6 @@ default long incrementCounter(String key) { */ void addMaximumSample(String key, long value); - /** - * Set a mean statistic to a given value. - *

- * No-op if the key is unknown. - *

- * @param key statistic key - * @param value new value. - */ - void setMeanStatistic(String key, MeanStatistic value); - /** * Add a sample to the mean statistics. *

diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitter.java index 024fb3ab34eb2..6216a1e1aeb6a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitter.java @@ -58,6 +58,8 @@ import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.logIOStatisticsAtDebug; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.CAPABILITY_DYNAMIC_PARTITIONING; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_DIAGNOSTICS_MANIFEST_DIR; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_IO_PROCESSORS; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_IO_PROCESSORS_DEFAULT; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_SUMMARY_REPORT_DIR; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_TASKS_COMPLETED_COUNT; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_TASKS_FAILED_COUNT; @@ -393,7 +395,9 @@ public void commitJob(final JobContext jobContext) throws IOException { marker = result.getJobSuccessData(); // update the cached success with the new report. setSuccessReport(marker); - + // patch in the #of threads as it is useful + marker.putDiagnostic(OPT_IO_PROCESSORS, + conf.get(OPT_IO_PROCESSORS, Long.toString(OPT_IO_PROCESSORS_DEFAULT))); } catch (IOException e) { // failure. record it for the summary failure = e; @@ -688,7 +692,7 @@ public String toString() { * to date. * The report will updated with the current active stage, * and if {@code thrown} is non-null, it will be added to the - * diagnistics (and the job tagged as a failure). + * diagnostics (and the job tagged as a failure). * Static for testability. * @param activeStage active stage * @param config configuration to use. diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/DiagnosticKeys.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/DiagnosticKeys.java index cb673383c58be..4c49b8bd6d2ea 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/DiagnosticKeys.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/DiagnosticKeys.java @@ -34,6 +34,9 @@ public final class DiagnosticKeys { public static final String STAGE = "stage"; public static final String EXCEPTION = "exception"; public static final String STACKTRACE = "stacktrace"; + public static final String TOTAL_MEMORY = "total.memory"; + public static final String FREE_MEMORY = "free.memory"; + public static final String HEAP_MEMORY = "heap.memory"; /** Directory where manifests were renamed: {@value}. */ diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestCommitterSupport.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestCommitterSupport.java index f6edde5f460b4..d75a2566778ce 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestCommitterSupport.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestCommitterSupport.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.time.ZonedDateTime; +import java.util.Map; import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.classification.InterfaceAudience; @@ -30,6 +31,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.statistics.IOStatisticsAggregator; +import org.apache.hadoop.fs.statistics.IOStatisticsSetters; import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.fs.statistics.impl.IOStatisticsStoreBuilder; import org.apache.hadoop.mapreduce.JobContext; @@ -53,12 +55,17 @@ import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.JOB_TASK_MANIFEST_SUBDIR; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.MANIFEST_COMMITTER_CLASSNAME; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.MANIFEST_SUFFIX; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_IO_PROCESSORS; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_IO_PROCESSORS_DEFAULT; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_STORE_OPERATIONS_CLASS; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SPARK_WRITE_UUID; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SUMMARY_FILENAME_FORMAT; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.TMP_SUFFIX; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DiagnosticKeys.FREE_MEMORY; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DiagnosticKeys.HEAP_MEMORY; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DiagnosticKeys.PRINCIPAL; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DiagnosticKeys.STAGE; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DiagnosticKeys.TOTAL_MEMORY; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.InternalConstants.COUNTER_STATISTICS; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.InternalConstants.DURATION_STATISTICS; @@ -224,6 +231,23 @@ public static ManifestSuccessData createManifestOutcome( return outcome; } + /** + * Add heap information to IOStatisticSetters gauges, with a stage in front of every key. + * @param ioStatisticsSetters map to update + * @param stage stage + */ + public static void addHeapInformation(IOStatisticsSetters ioStatisticsSetters, + String stage) { + // force a gc. bit of bad form but it makes for better numbers + System.gc(); + final long totalMemory = Runtime.getRuntime().totalMemory(); + final String prefix = "stage."; + ioStatisticsSetters.setGauge(prefix + stage + "." + TOTAL_MEMORY, totalMemory); + final long freeMemory = Runtime.getRuntime().freeMemory(); + ioStatisticsSetters.setGauge(prefix + stage + "." + FREE_MEMORY, freeMemory); + ioStatisticsSetters.setGauge(prefix + stage + "." + HEAP_MEMORY, totalMemory - freeMemory); + } + /** * Create the filename for a report from the jobID. * @param jobId jobId diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java index a754f2a2da8df..0427f6054670a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java @@ -28,6 +28,7 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; @@ -38,7 +39,11 @@ import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_BYTES_COMMITTED_COUNT; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_FILES_COMMITTED_COUNT; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_COMMIT; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_CREATE_TARGET_DIRS; +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_RENAME_FILES; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DiagnosticKeys.MANIFESTS; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.addHeapInformation; /** * Commit the Job. @@ -68,7 +73,8 @@ protected CommitJobStage.Result executeStage( storeSupportsResilientCommit()); boolean createMarker = arguments.isCreateMarker(); - + IOStatisticsSnapshot heapInfo = new IOStatisticsSnapshot(); + addHeapInformation(heapInfo, "setup"); // load the manifests final StageConfig stageConfig = getStageConfig(); LoadManifestsStage.Result result @@ -81,6 +87,7 @@ protected CommitJobStage.Result executeStage( getName(), summary.getFileCount(), byteCountToDisplaySize(summary.getTotalFileSize())); + addHeapInformation(heapInfo, OP_STAGE_JOB_LOAD_MANIFESTS); // add in the manifest statistics to our local IOStatistics for @@ -92,6 +99,7 @@ protected CommitJobStage.Result executeStage( final CreateOutputDirectoriesStage.Result dirStageResults = new CreateOutputDirectoriesStage(stageConfig) .apply(manifests); + addHeapInformation(heapInfo, OP_STAGE_JOB_CREATE_TARGET_DIRS); // commit all the tasks. // The success data includes a snapshot of the IO Statistics @@ -102,6 +110,8 @@ protected CommitJobStage.Result executeStage( if (LOG.isDebugEnabled()) { LOG.debug("{}: _SUCCESS file summary {}", getName(), successData.toJson()); } + addHeapInformation(heapInfo, OP_STAGE_JOB_RENAME_FILES); + // update the counter of bytes committed and files. // use setCounter so as to ignore any values accumulated when // aggregating tasks. @@ -112,6 +122,8 @@ protected CommitJobStage.Result executeStage( COMMITTER_BYTES_COMMITTED_COUNT, summary.getTotalFileSize()); successData.snapshotIOStatistics(iostats); + successData.getIOStatistics().aggregate(heapInfo); + // rename manifests. Only warn on failure here. diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java index 3720f549d89c3..c33f5248c516f 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java @@ -21,6 +21,8 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -31,6 +33,7 @@ import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; import org.apache.hadoop.fs.statistics.IOStatisticsSource; +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.util.functional.TaskPool; @@ -73,6 +76,11 @@ public class LoadManifestsStage extends */ private final List manifests = new ArrayList<>(); + /** + * Map of directories from manifests, coalesced to reduce duplication. + */ + private final Map directories = new ConcurrentHashMap<>(); + public LoadManifestsStage(final StageConfig stageConfig) { super(false, stageConfig, OP_STAGE_JOB_LOAD_MANIFESTS, true); } @@ -106,7 +114,7 @@ protected LoadManifestsStage.Result executeStage( // collect any stats maybeAddIOStatistics(getIOStatistics(), manifestFiles); - return new LoadManifestsStage.Result(summaryInfo, manifestList); + return new LoadManifestsStage.Result(summaryInfo, manifestList, directories); } /** @@ -152,6 +160,20 @@ private void processOneManifest(FileStatus status) } } + /** + * Coalesce all directories and clear the entry in the manifest. + * @param manifest manifest + */ + private void coalesceDirectories(TaskManifest manifest) { + final List destDirectories = manifest.getDestDirectories(); + synchronized (directories) { + destDirectories.forEach(entry -> { + directories.putIfAbsent(entry.getDir(), entry); + }); + } + manifest.getDestDirectories().clear(); + } + /** * Precommit preparation of a single manifest file. * To reduce the memory foot print, the IOStatistics and @@ -187,10 +209,13 @@ public static final class Result { private final List manifests; + private final Map directories; + public Result(SummaryInfo summary, - List manifests) { + List manifests, final Map directories) { this.summary = summary; this.manifests = manifests; + this.directories = directories; } public SummaryInfo getSummary() { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java index 526485df93490..0cb88fa83609e 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java @@ -756,66 +756,75 @@ private void testConcurrentCommitTaskWithSubDir(int version) conf.setInt(FileOutputCommitter.FILEOUTPUTCOMMITTER_ALGORITHM_VERSION, version); - conf.setClass("fs.file.impl", RLFS.class, FileSystem.class); + final String fileImpl = "fs.file.impl"; + final String fileImplClassname = "org.apache.hadoop.fs.LocalFileSystem"; + conf.setClass(fileImpl, RLFS.class, FileSystem.class); FileSystem.closeAll(); - final JobContext jContext = new JobContextImpl(conf, taskID.getJobID()); - final FileOutputCommitter amCommitter = - new FileOutputCommitter(outDir, jContext); - amCommitter.setupJob(jContext); - - final TaskAttemptContext[] taCtx = new TaskAttemptContextImpl[2]; - taCtx[0] = new TaskAttemptContextImpl(conf, taskID); - taCtx[1] = new TaskAttemptContextImpl(conf, taskID1); - - final TextOutputFormat[] tof = new TextOutputFormat[2]; - for (int i = 0; i < tof.length; i++) { - tof[i] = new TextOutputFormat() { - @Override - public Path getDefaultWorkFile(TaskAttemptContext context, - String extension) throws IOException { - final FileOutputCommitter foc = (FileOutputCommitter) - getOutputCommitter(context); - return new Path(new Path(foc.getWorkPath(), SUB_DIR), - getUniqueFile(context, getOutputName(context), extension)); - } - }; - } - - final ExecutorService executor = HadoopExecutors.newFixedThreadPool(2); try { - for (int i = 0; i < taCtx.length; i++) { - final int taskIdx = i; - executor.submit(new Callable() { + final JobContext jContext = new JobContextImpl(conf, taskID.getJobID()); + final FileOutputCommitter amCommitter = + new FileOutputCommitter(outDir, jContext); + amCommitter.setupJob(jContext); + + final TaskAttemptContext[] taCtx = new TaskAttemptContextImpl[2]; + taCtx[0] = new TaskAttemptContextImpl(conf, taskID); + taCtx[1] = new TaskAttemptContextImpl(conf, taskID1); + + final TextOutputFormat[] tof = new TextOutputFormat[2]; + for (int i = 0; i < tof.length; i++) { + tof[i] = new TextOutputFormat() { @Override - public Void call() throws IOException, InterruptedException { - final OutputCommitter outputCommitter = - tof[taskIdx].getOutputCommitter(taCtx[taskIdx]); - outputCommitter.setupTask(taCtx[taskIdx]); - final RecordWriter rw = - tof[taskIdx].getRecordWriter(taCtx[taskIdx]); - writeOutput(rw, taCtx[taskIdx]); - outputCommitter.commitTask(taCtx[taskIdx]); - return null; + public Path getDefaultWorkFile(TaskAttemptContext context, + String extension) throws IOException { + final FileOutputCommitter foc = (FileOutputCommitter) + getOutputCommitter(context); + return new Path(new Path(foc.getWorkPath(), SUB_DIR), + getUniqueFile(context, getOutputName(context), extension)); } - }); + }; } - } finally { - executor.shutdown(); - while (!executor.awaitTermination(1, TimeUnit.SECONDS)) { - LOG.info("Awaiting thread termination!"); + + final ExecutorService executor = HadoopExecutors.newFixedThreadPool(2); + try { + for (int i = 0; i < taCtx.length; i++) { + final int taskIdx = i; + executor.submit(new Callable() { + @Override + public Void call() throws IOException, InterruptedException { + final OutputCommitter outputCommitter = + tof[taskIdx].getOutputCommitter(taCtx[taskIdx]); + outputCommitter.setupTask(taCtx[taskIdx]); + final RecordWriter rw = + tof[taskIdx].getRecordWriter(taCtx[taskIdx]); + writeOutput(rw, taCtx[taskIdx]); + outputCommitter.commitTask(taCtx[taskIdx]); + return null; + } + }); + } + } finally { + executor.shutdown(); + while (!executor.awaitTermination(1, TimeUnit.SECONDS)) { + LOG.info("Awaiting thread termination!"); + } } - } - amCommitter.commitJob(jContext); - final RawLocalFileSystem lfs = new RawLocalFileSystem(); - lfs.setConf(conf); - assertFalse("Must not end up with sub_dir/sub_dir", - lfs.exists(new Path(OUT_SUB_DIR, SUB_DIR))); + amCommitter.commitJob(jContext); + final RawLocalFileSystem lfs = new RawLocalFileSystem(); + lfs.setConf(conf); + assertFalse("Must not end up with sub_dir/sub_dir", + lfs.exists(new Path(OUT_SUB_DIR, SUB_DIR))); - // validate output - validateContent(OUT_SUB_DIR); - FileUtil.fullyDelete(new File(outDir.toString())); + // validate output + validateContent(OUT_SUB_DIR); + FileUtil.fullyDelete(new File(outDir.toString())); + } finally { + // needed to avoid this test contaminating others in the same JVM + FileSystem.closeAll(); + conf.set(fileImpl, fileImplClassname); + conf.set(fileImpl, fileImplClassname); + } } @Test diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterTestSupport.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterTestSupport.java index 7297fdd174ec6..31abcb63f5368 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterTestSupport.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterTestSupport.java @@ -76,7 +76,7 @@ public final class ManifestCommitterTestSupport { * default number of task attempts for some tests. * Value: {@value}. */ - public static final int NUMBER_OF_TASK_ATTEMPTS = 200; + public static final int NUMBER_OF_TASK_ATTEMPTS = 2000; private ManifestCommitterTestSupport() { } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestLoadManifestsStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestLoadManifestsStage.java index 00372c5941efa..bf2b466310662 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestLoadManifestsStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestLoadManifestsStage.java @@ -18,19 +18,35 @@ package org.apache.hadoop.mapreduce.lib.output.committer.manifest; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.stream.Collectors; import org.assertj.core.api.Assertions; import org.junit.Test; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestPrinter; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.CleanupJobStage; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.CreateOutputDirectoriesStage; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.LoadManifestsStage; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.SetupJobStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.StageConfig; + +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_SUMMARY_REPORT_DIR; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_COMMIT; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.addHeapInformation; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.createJobSummaryFilename; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.createManifestOutcome; /** * Test loading manifests from a store. @@ -43,6 +59,8 @@ */ public class TestLoadManifestsStage extends AbstractManifestCommitterTest { + public static final int FILES_PER_TASK_ATTEMPT = 100; + private int taskAttemptCount; /** @@ -63,6 +81,10 @@ public void setup() throws Exception { .isGreaterThan(0); } + public long heapSize() { + return Runtime.getRuntime().totalMemory() - Runtime.getRuntime().freeMemory(); + } + /** * Build a large number of manifests, but without the real files * and directories. @@ -79,29 +101,40 @@ public void testSaveThenLoadManyManifests() throws Throwable { describe("Creating many manifests with fake file/dir entries," + " load them and prepare the output dirs."); - int filesPerTaskAttempt = 10; + int filesPerTaskAttempt = FILES_PER_TASK_ATTEMPT; LOG.info("Number of task attempts: {}, files per task attempt {}", taskAttemptCount, filesPerTaskAttempt); - setJobStageConfig(createStageConfigForJob(JOB1, getDestDir())); + final StageConfig stageConfig = createStageConfigForJob(JOB1, getDestDir()); + setJobStageConfig(stageConfig); // set up the job. - new SetupJobStage(getJobStageConfig()).apply(false); + new SetupJobStage(stageConfig).apply(false); LOG.info("Creating manifest files for {}", taskAttemptCount); executeTaskAttempts(taskAttemptCount, filesPerTaskAttempt); + IOStatisticsSnapshot heapInfo = new IOStatisticsSnapshot(); + addHeapInformation(heapInfo, "initial"); + LOG.info("Loading in the manifests"); // Load in the manifests LoadManifestsStage stage = new LoadManifestsStage( - getJobStageConfig()); + stageConfig); LoadManifestsStage.Result result = stage.apply(true); LoadManifestsStage.SummaryInfo summary = result.getSummary(); List loadedManifests = result.getManifests(); + LOG.info("\nJob statistics after loading {}", + ioStatisticsToPrettyString(getStageStatistics())); + LOG.info("Heap size = {}", heapSize()); + addHeapInformation(heapInfo, "load.manifests"); + + + Assertions.assertThat(summary.getManifestCount()) .describedAs("Manifest count of %s", summary) .isEqualTo(taskAttemptCount); @@ -112,6 +145,7 @@ public void testSaveThenLoadManyManifests() throws Throwable { .describedAs("File Size of %s", summary) .isEqualTo(getTotalDataSize()); + // now that manifest list. List manifestTaskIds = loadedManifests.stream() .map(TaskManifest::getTaskID) @@ -122,9 +156,10 @@ public void testSaveThenLoadManyManifests() throws Throwable { // now let's see about aggregating a large set of directories Set createdDirectories = new CreateOutputDirectoriesStage( - getJobStageConfig()) + stageConfig) .apply(loadedManifests) .getCreatedDirectories(); + addHeapInformation(heapInfo, "create.directories"); // but after the merge process, only one per generated file output // dir exists @@ -134,8 +169,26 @@ public void testSaveThenLoadManyManifests() throws Throwable { // and skipping the rename stage (which is going to fail), // go straight to cleanup - new CleanupJobStage(getJobStageConfig()).apply( + new CleanupJobStage(stageConfig).apply( new CleanupJobStage.Arguments("", true, true, false)); + addHeapInformation(heapInfo, "cleanup"); + + ManifestSuccessData success = createManifestOutcome(stageConfig, OP_STAGE_JOB_COMMIT); + success.snapshotIOStatistics(getStageStatistics()); + success.getIOStatistics().aggregate(heapInfo); + + Configuration conf = getConfiguration(); + enableManifestCommitter(conf); + String reportDir = conf.getTrimmed(OPT_SUMMARY_REPORT_DIR, ""); + Path reportDirPath = new Path(reportDir); + Path path = new Path(reportDirPath, + createJobSummaryFilename("TestLoadManifestsStage")); + final FileSystem summaryFS = path.getFileSystem(conf); + success.save(summaryFS, path, true); + LOG.info("Saved summary to {}", path); + ManifestPrinter showManifest = new ManifestPrinter(); + ManifestSuccessData manifestSuccessData = + showManifest.loadAndPrintManifest(summaryFS, path); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/AbfsCommitTestHelper.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/AbfsCommitTestHelper.java index 8160cdc64c546..37dcfd7c45148 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/AbfsCommitTestHelper.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/AbfsCommitTestHelper.java @@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.azurebfs.contract.ABFSContractTestBinding; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_READ_SMALL_FILES_COMPLETELY; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_STORE_OPERATIONS_CLASS; /** @@ -43,6 +44,8 @@ static Configuration prepareTestConfiguration( // use ABFS Store operations conf.set(OPT_STORE_OPERATIONS_CLASS, AbfsManifestStoreOperations.NAME); + // turn on small file read if not explicitly set to a value. + conf.setBooleanIfUnset(AZURE_READ_SMALL_FILES_COMPLETELY, true); return conf; } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsLoadManifestsStage.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsLoadManifestsStage.java index acd693e39a0f0..8efdc7fb6f687 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsLoadManifestsStage.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsLoadManifestsStage.java @@ -19,6 +19,7 @@ package org.apache.hadoop.fs.azurebfs.commit; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azure.integration.AzureTestConstants; import org.apache.hadoop.fs.azurebfs.contract.ABFSContractTestBinding; import org.apache.hadoop.fs.azurebfs.contract.AbfsFileSystemContract; import org.apache.hadoop.fs.contract.AbstractFSContract; @@ -52,4 +53,9 @@ protected AbstractFSContract createContract(final Configuration conf) { return new AbfsFileSystemContract(conf, binding.isSecureMode()); } + @Override + protected int getTestTimeoutMillis() { + return AzureTestConstants.SCALE_TEST_TIMEOUT_MILLIS; + } + } From e153791187f69b8f130d08c00e879e1d79f4a255 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 31 Mar 2023 13:26:08 +0100 Subject: [PATCH 02/18] MAPREDUCE-7435. committer OOM * DirEntry and FileEntry are writeable * LoadManifests to take a path to where to cache the rename list. not yet wired up. Change-Id: Ibd992b179bd0bcf26a39ae4ce5407257ecbfcb10 --- .../committer/manifest/files/DirEntry.java | 26 +++++++++-- .../committer/manifest/files/FileEntry.java | 23 +++++++++- .../manifest/stages/CommitJobStage.java | 6 ++- .../manifest/stages/LoadManifestsStage.java | 43 ++++++++++++++++--- .../TestJobThroughManifestCommitter.java | 4 +- 5 files changed, 88 insertions(+), 14 deletions(-) diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/DirEntry.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/DirEntry.java index 15e8cac779592..994ead3472f1b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/DirEntry.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/DirEntry.java @@ -18,6 +18,8 @@ package org.apache.hadoop.mapreduce.lib.output.committer.manifest.files; +import java.io.DataInput; +import java.io.DataOutput; import java.io.IOException; import java.io.Serializable; import java.util.Objects; @@ -28,6 +30,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.Writable; import static java.util.Objects.requireNonNull; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData.marshallPath; @@ -37,12 +40,13 @@ /** * A directory entry in the task manifest. * Uses shorter field names for smaller files. - * Hash and equals are on dir name only; there's no real expectation - * that those operations are needed. + * Hash and equals are on dir name only. + * Can be serialized as a java object, json object + * or hadoop writable. */ @InterfaceAudience.Public @InterfaceStability.Unstable -public final class DirEntry implements Serializable { +public final class DirEntry implements Serializable, Writable { private static final long serialVersionUID = 5658520530209859765L; @@ -65,7 +69,7 @@ public final class DirEntry implements Serializable { private int level; /** - * Constructor only for use by jackson. + * Constructor for use by jackson/writable. * Do Not Delete. */ private DirEntry() { @@ -177,6 +181,20 @@ public int hashCode() { return Objects.hash(dir); } + @Override + public void write(final DataOutput out) throws IOException { + out.writeUTF(dir); + out.writeInt(type); + out.writeInt(level); + } + + @Override + public void readFields(final DataInput in) throws IOException { + dir = in.readUTF(); + type = in.readInt(); + level = in.readInt(); + } + /** * A directory entry. * @param dest destination path. diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/FileEntry.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/FileEntry.java index bc6cdd94b3f26..ac6664262378b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/FileEntry.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/FileEntry.java @@ -18,6 +18,8 @@ package org.apache.hadoop.mapreduce.lib.output.committer.manifest.files; +import java.io.DataInput; +import java.io.DataOutput; import java.io.IOException; import java.io.Serializable; import java.util.Objects; @@ -29,6 +31,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.Writable; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData.marshallPath; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData.unmarshallPath; @@ -42,7 +45,7 @@ @InterfaceAudience.Private @InterfaceStability.Unstable @JsonInclude(JsonInclude.Include.NON_NULL) -public final class FileEntry implements Serializable { +public final class FileEntry implements Serializable, Writable { private static final long serialVersionUID = -550288489009777867L; @@ -62,7 +65,7 @@ public final class FileEntry implements Serializable { private String etag; /** - * Constructor only for use by jackson. + * Constructor for use by jackson/writable * Do Not Delete. */ private FileEntry() { @@ -181,6 +184,22 @@ public boolean equals(Object o) { Objects.equals(etag, that.etag); } + @Override + public void write(final DataOutput out) throws IOException { + out.writeUTF(source); + out.writeUTF(dest); + out.writeUTF(etag); + out.writeLong(size); + } + + @Override + public void readFields(final DataInput in) throws IOException { + source = in.readUTF(); + dest = in.readUTF(); + etag = in.readUTF(); + size = in.readLong(); + } + @Override public int hashCode() { return Objects.hash(source, dest); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java index 0427f6054670a..290e55e9fdee5 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java @@ -18,6 +18,7 @@ package org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages; +import java.io.File; import java.io.IOException; import java.util.List; @@ -77,8 +78,9 @@ protected CommitJobStage.Result executeStage( addHeapInformation(heapInfo, "setup"); // load the manifests final StageConfig stageConfig = getStageConfig(); - LoadManifestsStage.Result result - = new LoadManifestsStage(stageConfig).apply(true); + LoadManifestsStage.Result result = new LoadManifestsStage(stageConfig).apply( + new LoadManifestsStage.Arguments( + File.createTempFile("manifest", ".list"), false)); List manifests = result.getManifests(); LoadManifestsStage.SummaryInfo summary = result.getSummary(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java index c33f5248c516f..17d9c56baf619 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java @@ -18,6 +18,7 @@ package org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages; +import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -55,7 +56,7 @@ */ public class LoadManifestsStage extends AbstractJobOrTaskStage< - Boolean, + LoadManifestsStage.Arguments, LoadManifestsStage.Result> { private static final Logger LOG = LoggerFactory.getLogger( @@ -93,13 +94,13 @@ public LoadManifestsStage(final StageConfig stageConfig) { */ @Override protected LoadManifestsStage.Result executeStage( - final Boolean prune) throws IOException { + final LoadManifestsStage.Arguments arguments) throws IOException { final Path manifestDir = getTaskManifestDir(); LOG.info("{}: Executing Manifest Job Commit with manifests in {}", getName(), manifestDir); - pruneManifests = prune; + pruneManifests = true; // build a list of all task manifests successfully committed // msync(manifestDir); @@ -114,7 +115,7 @@ protected LoadManifestsStage.Result executeStage( // collect any stats maybeAddIOStatistics(getIOStatistics(), manifestFiles); - return new LoadManifestsStage.Result(summaryInfo, manifestList, directories); + return new LoadManifestsStage.Result(summaryInfo, null, manifestList, directories); } /** @@ -201,19 +202,51 @@ private TaskManifest fetchTaskManifest(FileStatus status) return manifest; } + /** + * Stage arguments. + */ + public static final class Arguments { + /** + * File where the listing has been saved. + */ + private final File renameListFile; + + /** + * build a list of manifests and return them? + */ + private final boolean cacheManifests; + + public Arguments(final File renameListFile, final boolean cacheManifests) { + this.renameListFile = renameListFile; + this.cacheManifests = cacheManifests; + } + } + /** * Result of the stage. */ public static final class Result { private final SummaryInfo summary; + /** + * File where the listing has been saved. + */ + private final File renameListFile; + + /** + * manifest list, non-null only if cacheManifests is true. + */ private final List manifests; + /** + * Map of directories. + */ private final Map directories; public Result(SummaryInfo summary, - List manifests, final Map directories) { + final File renameListFile, List manifests, final Map directories) { this.summary = summary; + this.renameListFile = renameListFile; this.manifests = manifests; this.directories = directories; } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java index fa3527f11cc2b..4996c2031fa0b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java @@ -18,6 +18,7 @@ package org.apache.hadoop.mapreduce.lib.output.committer.manifest; +import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; import java.util.List; @@ -482,7 +483,8 @@ public void test_0420_validateJob() throws Throwable { // load manifests stage will load all the task manifests again List manifests = new LoadManifestsStage(getJobStageConfig()) - .apply(true).getManifests(); + .apply(new LoadManifestsStage.Arguments( + File.createTempFile("manifest", ".list"), true)).getManifests(); // Now verify their files exist, returning the list of renamed files. List committedFiles = new ValidateRenamedFilesStage(getJobStageConfig()) .apply(manifests) From 16422e498c6222264f2d77778e024848dc8150cb Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 4 Apr 2023 21:35:24 +0100 Subject: [PATCH 03/18] MAPREDUCE-7435. oom: switch to sequence file for storage of the files. This is a big change, with tests as far as verifying core read/write happy. Current state: simple read/write good, async queue not yet tested Change-Id: I7cb1443024780b355a8f3bb96fbfe08d8608d968 --- .../impl/IOStatisticsStoreBuilder.java | 11 + .../impl/IOStatisticsStoreBuilderImpl.java | 12 + .../util/functional/RemoteIterators.java | 57 ++- .../manifest/ManifestCommitterConfig.java | 1 + .../committer/manifest/files/FileEntry.java | 40 ++- .../committer/manifest/impl/EntryFileIO.java | 329 ++++++++++++++++++ .../manifest/impl/InternalConstants.java | 13 + .../manifest/impl/LoadedManifestData.java | 72 ++++ .../impl/ManifestCommitterSupport.java | 5 +- .../manifest/stages/CommitJobStage.java | 8 +- .../stages/CreateOutputDirectoriesStage.java | 76 ++-- .../manifest/stages/LoadManifestsStage.java | 199 ++++++++--- .../manifest/stages/StageConfig.java | 26 ++ .../AbstractManifestCommitterTest.java | 1 + .../TestCreateOutputDirectoriesStage.java | 39 +-- .../TestJobThroughManifestCommitter.java | 12 +- .../manifest/TestLoadManifestsStage.java | 39 ++- .../manifest/impl/TestEntryFileIO.java | 95 +++++ 18 files changed, 866 insertions(+), 169 deletions(-) create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/LoadedManifestData.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/TestEntryFileIO.java 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 index d94a8389b7ff8..f1272d53ebbb2 100644 --- 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 @@ -67,6 +67,17 @@ public interface IOStatisticsStoreBuilder { IOStatisticsStoreBuilder withDurationTracking( String... prefixes); + /** + * A value which is tracked with counter/min/max/mean. + * Similar to {@link #withDurationTracking(String...)} + * but without the failure option and with the same name + * across all categories. + * @param prefixes prefixes to add. + * @return the builder + */ + IOStatisticsStoreBuilder withSampleTracking( + String... prefixes); + /** * Build the collector. * @return a new collector. 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 index 0562271db3ef8..70d4f6951d3d2 100644 --- 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 @@ -92,6 +92,18 @@ public IOStatisticsStoreBuilderImpl withDurationTracking( return this; } + @Override + public IOStatisticsStoreBuilderImpl withSampleTracking( + final String... prefixes) { + for (String p : prefixes) { + withCounters(p); + withMinimums(p); + withMaximums(p); + withMeanStatistics(p); + } + return this; + } + @Override public IOStatisticsStore build() { return new IOStatisticsStoreImpl(counters, gauges, minimums, 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 index d2f7742d3d988..f82c7e281ea61 100644 --- 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 @@ -191,6 +191,20 @@ public static RemoteIterator closingRemoteIterator( return new CloseRemoteIterator<>(iterator, toClose); } + /** + * Wrap an iterator with one which adds a continuation probe. + * This allows work to exit fast without complicated breakout logic + * @param iterator source + * @param continueWork predicate which will trigger a fast halt if it returns false. + * @param source type. + * @return a new iterator + */ + public static RemoteIterator haltableRemoteIterator( + final RemoteIterator iterator, + final CallableRaisingIOE continueWork) { + return new HaltableRemoteIterator<>(iterator, continueWork); + } + /** * Build a list from a RemoteIterator. * @param source source iterator @@ -391,10 +405,12 @@ public void close() throws IOException { /** * Wrapper of another remote iterator; IOStatistics * and Closeable methods are passed down if implemented. + * This class may be subclasses if custom iterators + * are needed. * @param source type * @param type of returned value */ - private static abstract class WrappingRemoteIterator + public static abstract class WrappingRemoteIterator implements RemoteIterator, IOStatisticsSource, Closeable { /** @@ -715,4 +731,43 @@ public void close() throws IOException { } } } + + /** + * An iterator which allows for a fast exit predicate. + * @param source type + */ + private static final class HaltableRemoteIterator + extends WrappingRemoteIterator { + + private final CallableRaisingIOE continueWork; + + + /** + * Wrap an iterator with one which adds a continuation probe. + * @param source source iterator. + * @param continueWork predicate which will trigger a fast halt if it returns false. + */ + private HaltableRemoteIterator( + final RemoteIterator source, + final CallableRaisingIOE continueWork) { + super(source); + this.continueWork = continueWork; + } + + @Override + public boolean hasNext() throws IOException { + return sourceHasNext(); + } + + @Override + public S next() throws IOException { + return sourceNext(); + } + + @Override + protected boolean sourceHasNext() throws IOException { + return continueWork.apply() && super.sourceHasNext(); + } + } + } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConfig.java index 40dffee203b8d..eb9c502129f72 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConfig.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConfig.java @@ -251,6 +251,7 @@ FileSystem getDestinationFileSystem() throws IOException { StageConfig createStageConfig() { StageConfig stageConfig = new StageConfig(); stageConfig + .withConfiguration(conf) .withIOStatistics(iostatistics) .withJobAttemptNumber(jobAttemptNumber) .withJobDirectories(dirs) diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/FileEntry.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/FileEntry.java index ac6664262378b..ac4498d6d9d8b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/FileEntry.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/FileEntry.java @@ -31,7 +31,9 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; +import org.apache.hadoop.io.WritableUtils; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData.marshallPath; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData.unmarshallPath; @@ -40,6 +42,8 @@ /** * A File entry in the task manifest. * Uses shorter field names for smaller files. + * Used as a Hadoop writable when saved to in intermediate file + * during job commit. */ @InterfaceAudience.Private @@ -65,10 +69,10 @@ public final class FileEntry implements Serializable, Writable { private String etag; /** - * Constructor for use by jackson/writable + * Constructor for serialization/deserialization. * Do Not Delete. */ - private FileEntry() { + public FileEntry() { } /** @@ -179,30 +183,30 @@ public boolean equals(Object o) { return false; } FileEntry that = (FileEntry) o; - return size == that.size && source.equals(that.source) && dest.equals( - that.dest) && - Objects.equals(etag, that.etag); + return size == that.size + && Objects.equals(source, that.source) + && Objects.equals(dest, that.dest) + && Objects.equals(etag, that.etag); } @Override - public void write(final DataOutput out) throws IOException { - out.writeUTF(source); - out.writeUTF(dest); - out.writeUTF(etag); - out.writeLong(size); + public int hashCode() { + return Objects.hash(source, dest); } @Override - public void readFields(final DataInput in) throws IOException { - source = in.readUTF(); - dest = in.readUTF(); - etag = in.readUTF(); - size = in.readLong(); + public void write(final DataOutput out) throws IOException { + Text.writeString(out, source); + Text.writeString(out, dest); + Text.writeString(out, etag); + WritableUtils.writeVLong(out, size); } @Override - public int hashCode() { - return Objects.hash(source, dest); + public void readFields(final DataInput in) throws IOException { + source = Text.readString(in); + dest = Text.readString(in); + etag = Text.readString(in); + size = WritableUtils.readVLong(in); } - } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java new file mode 100644 index 0000000000000..efa561b06405a --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java @@ -0,0 +1,329 @@ +/* + * 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.mapreduce.lib.output.committer.manifest.impl; + +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.ArrayList; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Queue; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry; +import org.apache.hadoop.util.Preconditions; +import org.apache.hadoop.util.concurrent.HadoopExecutors; +import org.apache.hadoop.util.functional.FutureIO; + +import static java.util.Objects.requireNonNull; + +/** + * Read or write entry file. + * This can be used to create a simple reader, or to create + * a writer queue where different threads can queue data for + * writing. + * The entry file is a SequenceFile with KV = {NullWritable, FileEntry}; + */ +public class EntryFileIO { + + private static final Logger LOG = LoggerFactory.getLogger( + EntryFileIO.class); + + /** Configuration used to load filesystems. */ + private final Configuration conf; + + /** + * ctor. + * @param conf Configuration used to load filesystems + */ + public EntryFileIO(final Configuration conf) { + this.conf = conf; + } + + /** + * Create a writer to a local file. + * @param file file + * @return the writer + * @throws IOException fail to open the file + */ + public SequenceFile.Writer createWriter(File file) throws IOException { + return createWriter(new Path(file.toURI())); + } + + public SequenceFile.Writer createWriter(Path path) throws IOException { + return SequenceFile.createWriter(conf, + SequenceFile.Writer.file(path), + SequenceFile.Writer.keyClass(NullWritable.class), + SequenceFile.Writer.valueClass(FileEntry.class)); + } + + + /** + * Reader is created with sequential reads. + * @param file file + * @return the reader + * @throws IOException failure to open + */ + public SequenceFile.Reader createReader(File file) throws IOException { + return createReader(new Path(file.toURI())); + } + + /** + * Reader is created with sequential reads. + * @param path path + * @return the reader + * @throws IOException failure to open + */ + public SequenceFile.Reader createReader(Path path) throws IOException { + return new SequenceFile.Reader(conf, + SequenceFile.Reader.file(path)); + } + + /** + * Iterator to retrieve file entries from the sequence file. + * Closeable: cast and invoke to close the reader. + * @param reader reader; + * @return iterator + */ + public RemoteIterator iterateOver(SequenceFile.Reader reader) { + return new EntryIterator(reader); + } + + /** + * Create and start an entry writer. + * @param writer writer + * @param capacity queue capacity + * @return the writer. + */ + public EntryWriter launchEntryWriter(SequenceFile.Writer writer, int capacity) { + final EntryWriter ew = new EntryWriter(writer, capacity); + ew.start(); + return ew; + } + + /** + * Writer takes a list of entries at a time; queues for writing. + * A special + */ + public final class EntryWriter implements Closeable { + + private final SequenceFile.Writer writer; + + private final Queue> queue; + + /** + * stop flag. + */ + private final AtomicBoolean stop = new AtomicBoolean(false); + + private final AtomicBoolean active = new AtomicBoolean(false); + + /** + * Executor of writes. + */ + private ExecutorService executor; + + /** + * Future invoked. + */ + private Future future; + + /** + * count of files opened; only updated in one thread + * so volatile. + */ + private volatile int count; + + /** + * any failure. + */ + private volatile IOException failure; + + /** + * Create. + * @param writer writer + * @param capacity capacity. + */ + private EntryWriter(SequenceFile.Writer writer, int capacity) { + this.writer = writer; + this.queue = new ArrayBlockingQueue<>(capacity); + } + + /** + * Is the writer active? + * @return true if the processor thread is live + */ + public boolean isActive() { + return active.get(); + } + + /** + * Get count of files processed. + * @return the count + */ + public int getCount() { + return count; + } + + /** + * Any failure. + * @return any IOException caught when writing the output + */ + public IOException getFailure() { + return failure; + } + + /** + * Start the thread. + */ + private void start() { + Preconditions.checkState(executor == null, "already started"); + active.set(true); + executor = HadoopExecutors.newSingleThreadExecutor(); + future = executor.submit(this::processor); + } + + /** + * Add a list of entries to the queue. + * @param entries entries. + * @return whether the queue worked. + */ + public boolean enqueue(List entries) { + if (active.get()) { + queue.add(entries); + return false; + } else { + LOG.debug("Queue inactive; discarding {} entries", entries.size()); + return false; + } + } + + /** + * Queue and process entries until done. + * @return count of entries written. + * @throws UncheckedIOException on write failure + */ + private int processor() { + int count = 0; + while (!stop.get()) { + queue.poll().forEach(this::append); + } + return count; + } + + /** + * write one entry. + * @param entry entry to write + * @throws UncheckedIOException on write failure + */ + private void append(FileEntry entry) { + if (failure != null) { + try { + writer.append(NullWritable.get(), entry); + count++; + } catch (IOException e) { + failure = e; + throw new UncheckedIOException(e); + } + } + } + + @Override + public void close() throws IOException { + if (stop.getAndSet(true)) { + // already stopped + return; + } + LOG.debug("Shutting down writer"); + // signal queue closure by + // clearing the current list + // and queue an empty list + queue.clear(); + queue.add(new ArrayList<>()); + try { + // wait for the op to finish. + final int count = FutureIO.awaitFuture(future); + LOG.debug("Processed {} files", count); + // close the stream + } finally { + writer.close(); + } + } + } + + /** + * Iterator to retrieve file entries from the sequence file. + * Closeable. + */ + private final class EntryIterator implements RemoteIterator, Closeable { + + private final SequenceFile.Reader reader; + + private FileEntry fetched; + + private EntryIterator(final SequenceFile.Reader reader) { + this.reader = requireNonNull(reader); + } + + @Override + public void close() throws IOException { + reader.close(); + } + + @Override + public boolean hasNext() throws IOException { + return fetched != null || fetchNext(); + } + + private boolean fetchNext() throws IOException { + FileEntry readBack = new FileEntry(); + if (reader.next(NullWritable.get(), readBack)) { + fetched = readBack; + return true; + } else { + fetched = null; + return false; + } + } + + @Override + public FileEntry next() throws IOException { + if (!hasNext()) { + throw new NoSuchElementException(); + } + final FileEntry r = fetched; + fetched = null; + return r; + } + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/InternalConstants.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/InternalConstants.java index 15f9899f3551e..804cbde01f36f 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/InternalConstants.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/InternalConstants.java @@ -40,6 +40,7 @@ */ @InterfaceAudience.Private public final class InternalConstants { + private InternalConstants() { } @@ -127,4 +128,16 @@ private InternalConstants() { /** Schemas of filesystems we know to not work with this committer. */ public static final Set UNSUPPORTED_FS_SCHEMAS = ImmutableSet.of("s3a", "wasb"); + + /** + * Queue capacity between task manifest loading an entry file writer. + * If more than this number of manifest lists are waiting to be written, + * the enqueue is blocking. + * There's an expectation that writing to the local file is a lot faster + * than the parallelized buffer reads, therefore that this queue can + * be emptied at the same rate it is filled. + * Value {@value}. + */ + public static final int ENTRY_WRITER_QUEUE_CAPACITY = 32; + } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/LoadedManifestData.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/LoadedManifestData.java new file mode 100644 index 0000000000000..1aed6d1fcbd5c --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/LoadedManifestData.java @@ -0,0 +1,72 @@ +/* + * 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.mapreduce.lib.output.committer.manifest.impl; + +import java.util.Collection; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DirEntry; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.LoadManifestsStage; + +/** + * Information about the loaded manifest data; + * Returned from {@link LoadManifestsStage} and then + * used for renaming the work. + */ +public final class LoadedManifestData { + + /** + * Directories. + */ + private final Collection directories; + + /** + * Path of the intermediate cache of + * files to rename. + * This will be a sequence file of long -> FileEntry + */ + private final Path entrySequenceFile; + + /** + * How many files will be renamed. + */ + private final int fileCount; + + public LoadedManifestData( + final Collection directories, + final Path entrySequenceFile, + final int fileCount) { + this.directories = directories; + this.fileCount = fileCount; + this.entrySequenceFile = entrySequenceFile; + } + + public Collection getDirectories() { + return directories; + } + + public int getFileCount() { + return fileCount; + } + + public Path getEntrySequenceFile() { + return entrySequenceFile; + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestCommitterSupport.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestCommitterSupport.java index d75a2566778ce..c8ee77c84586d 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestCommitterSupport.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestCommitterSupport.java @@ -89,10 +89,7 @@ public static IOStatisticsStoreBuilder createIOStatisticsStore() { final IOStatisticsStoreBuilder store = iostatisticsStore(); - store.withCounters(COUNTER_STATISTICS); - store.withMaximums(COUNTER_STATISTICS); - store.withMinimums(COUNTER_STATISTICS); - store.withMeanStatistics(COUNTER_STATISTICS); + store.withSampleTracking(COUNTER_STATISTICS); store.withDurationTracking(DURATION_STATISTICS); return store; } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java index 290e55e9fdee5..c8ead7be4e669 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java @@ -33,6 +33,7 @@ import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.LoadedManifestData; import static java.util.Objects.requireNonNull; import static org.apache.commons.io.FileUtils.byteCountToDisplaySize; @@ -44,6 +45,7 @@ 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_RENAME_FILES; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DiagnosticKeys.MANIFESTS; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.InternalConstants.ENTRY_WRITER_QUEUE_CAPACITY; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.addHeapInformation; /** @@ -80,9 +82,11 @@ protected CommitJobStage.Result executeStage( final StageConfig stageConfig = getStageConfig(); LoadManifestsStage.Result result = new LoadManifestsStage(stageConfig).apply( new LoadManifestsStage.Arguments( - File.createTempFile("manifest", ".list"), false)); + File.createTempFile("manifest", ".list"), false, + ENTRY_WRITER_QUEUE_CAPACITY)); List manifests = result.getManifests(); LoadManifestsStage.SummaryInfo summary = result.getSummary(); + final LoadedManifestData manifestData = result.getLoadedManifestData(); LOG.debug("{}: Job Summary {}", getName(), summary); LOG.info("{}: Committing job with file count: {}; total size {} bytes", @@ -100,7 +104,7 @@ protected CommitJobStage.Result executeStage( // prepare destination directories. final CreateOutputDirectoriesStage.Result dirStageResults = new CreateOutputDirectoriesStage(stageConfig) - .apply(manifests); + .apply(manifestData.getDirectories()); addHeapInformation(heapInfo, OP_STAGE_JOB_CREATE_TARGET_DIRS); // commit all the tasks. diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CreateOutputDirectoriesStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CreateOutputDirectoriesStage.java index 927004e619742..1b717886c71de 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CreateOutputDirectoriesStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CreateOutputDirectoriesStage.java @@ -21,7 +21,9 @@ import java.io.IOException; import java.time.Duration; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -37,7 +39,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DirEntry; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.EntryStatus; -import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; import org.apache.hadoop.util.functional.TaskPool; import static java.util.Objects.requireNonNull; @@ -75,16 +76,14 @@ */ public class CreateOutputDirectoriesStage extends AbstractJobOrTaskStage< - List, + Collection, CreateOutputDirectoriesStage.Result> { private static final Logger LOG = LoggerFactory.getLogger( CreateOutputDirectoriesStage.class); /** - * Directories as a map of (path, path). - * Using a map rather than any set for efficient concurrency; the - * concurrent sets don't do lookups so fast. + * Directories as a map of (path, DirMapState). */ private final Map dirMap = new ConcurrentHashMap<>(); @@ -101,20 +100,20 @@ public CreateOutputDirectoriesStage(final StageConfig stageConfig) { @Override protected Result executeStage( - final List taskManifests) + final Collection manifestDirs) throws IOException { - final List directories = createAllDirectories(taskManifests); + final List directories = createAllDirectories(manifestDirs); LOG.debug("{}: Created {} directories", getName(), directories.size()); return new Result(new HashSet<>(directories), dirMap); } /** - * For each task, build the list of directories it wants. - * @param taskManifests task manifests + * Build the list of directories to create. + * @param manifestDirs dir entries from the manifests * @return the list of paths which have been created. */ - private List createAllDirectories(final List taskManifests) + private List createAllDirectories(final Collection manifestDirs) throws IOException { // all directories which need to exist across all @@ -128,32 +127,27 @@ private List createAllDirectories(final List taskManifests) // will be created at that path. final Set filesToDelete = new HashSet<>(); - // iterate through the task manifests - // and all output dirs into the set of dirs to - // create. - // hopefully there is a lot of overlap, so the - // final number of dirs to create is small. - for (TaskManifest task: taskManifests) { - final List destDirectories = task.getDestDirectories(); - Collections.sort(destDirectories, (o1, o2) -> - o1.getLevel() - o2.getLevel()); - for (DirEntry entry: destDirectories) { - // add the dest entry - final Path path = entry.getDestPath(); - if (!leaves.containsKey(path)) { - leaves.put(path, entry); - - // if it is a file to delete, record this. - if (entry.getStatus() == EntryStatus.file) { - filesToDelete.add(path); - } - final Path parent = path.getParent(); - if (parent != null && leaves.containsKey(parent)) { - // there's a parent dir, move it from the leaf list - // to parent list - parents.put(parent, - leaves.remove(parent)); - } + // sort the values of dir map by directory level: parent dirs will + // come first in the sorting + List destDirectories = new ArrayList<>(manifestDirs); + + Collections.sort(destDirectories, Comparator.comparingInt(DirEntry::getLevel)); + // iterate through the directory map + for (DirEntry entry: destDirectories) { + // add the dest entry + final Path path = entry.getDestPath(); + if (!leaves.containsKey(path)) { + leaves.put(path, entry); + + // if it is a file to delete, record this. + if (entry.getStatus() == EntryStatus.file) { + filesToDelete.add(path); + } + final Path parent = path.getParent(); + if (parent != null && leaves.containsKey(parent)) { + // there's a parent dir, move it from the leaf list + // to parent list + parents.put(parent, leaves.remove(parent)); } } } @@ -168,7 +162,9 @@ private List createAllDirectories(final List taskManifests) // Now the real work. final int createCount = leaves.size(); - LOG.info("Preparing {} directory/directories", createCount); + LOG.info("Preparing {} directory/directories; {} parent dirs implicitly created", + createCount, parents.size()); + // now probe for and create the leaf dirs, which are those at the // bottom level Duration d = measureDurationOfInvocation(getIOStatistics(), OP_CREATE_DIRECTORIES, () -> @@ -188,7 +184,7 @@ private List createAllDirectories(final List taskManifests) /** * report a single directory failure. - * @param path path which could not be deleted + * @param dirEntry dir which could not be deleted * @param e exception raised. */ private void reportMkDirFailure(DirEntry dirEntry, Exception e) { @@ -274,8 +270,8 @@ private void createOneDirectory(final DirEntry dirEntry) throws IOException { * Try to efficiently and robustly create a directory in a method which is * expected to be executed in parallel with operations creating * peer directories. - * @param path path to create - * @return true if dir created/found + * @param dirEntry dir to create + * @return Outcome * @throws IOException IO Failure. */ private DirMapState maybeCreateOneDirectory(DirEntry dirEntry) throws IOException { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java index 17d9c56baf619..8af2a38b6fbb7 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java @@ -24,6 +24,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -34,17 +35,23 @@ import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; 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.impl.EntryFileIO; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.LoadedManifestData; import org.apache.hadoop.util.functional.TaskPool; import static org.apache.commons.io.FileUtils.byteCountToDisplaySize; import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics; 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.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.impl.ManifestCommitterSupport.maybeAddIOStatistics; +import static org.apache.hadoop.util.functional.RemoteIterators.haltableRemoteIterator; /** * Stage to load all the task manifests in the job attempt directory. @@ -67,10 +74,8 @@ public class LoadManifestsStage extends */ private final SummaryInfo summaryInfo = new SummaryInfo(); - /** - * Should manifests be pruned of IOStatistics? - */ - private boolean pruneManifests; + + /** * List of loaded manifests. @@ -82,13 +87,24 @@ public class LoadManifestsStage extends */ private final Map directories = new ConcurrentHashMap<>(); + /** + * Writer of entries. + */ + private EntryFileIO.EntryWriter entryWriter; + + /** + * Should the manifests be cached and returned? + * only for testing. + */ + private boolean cacheManifests; + public LoadManifestsStage(final StageConfig stageConfig) { super(false, stageConfig, OP_STAGE_JOB_LOAD_MANIFESTS, true); } /** * Load the manifests. - * @param prune should manifests be pruned of IOStatistics? + * @param arguments stage arguments * @return the summary and a list of manifests. * @throws IOException IO failure. */ @@ -96,26 +112,51 @@ public LoadManifestsStage(final StageConfig stageConfig) { protected LoadManifestsStage.Result executeStage( final LoadManifestsStage.Arguments arguments) throws IOException { + EntryFileIO entryFileIO = new EntryFileIO(getStageConfig().getConf()); + final Path manifestDir = getTaskManifestDir(); LOG.info("{}: Executing Manifest Job Commit with manifests in {}", getName(), manifestDir); - pruneManifests = true; - // build a list of all task manifests successfully committed - // - msync(manifestDir); - final RemoteIterator manifestFiles = listManifests(); - - final List manifestList = loadAllManifests(manifestFiles); - LOG.info("{}: Summary of {} manifests loaded in {}: {}", - getName(), - manifestList.size(), - manifestDir, - summaryInfo); + cacheManifests = arguments.cacheManifests; + + final Path entrySequenceFile = arguments.getEntrySequenceFile(); + + // the entry writer for queuing data. + entryWriter = entryFileIO.launchEntryWriter( + entryFileIO.createWriter(entrySequenceFile), + arguments.queueCapacity); + try { + + // sync fs before the listj + 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 manifestFiles = + haltableRemoteIterator(listManifests(), () -> entryWriter.isActive()); + + final List manifestList = loadAllManifests(manifestFiles); + LOG.info("{}: Summary of {} manifests loaded in {}: {}", + getName(), + manifestList.size(), + manifestDir, + summaryInfo); + + // close cleanly + entryWriter.close(); + + // collect any stats + maybeAddIOStatistics(getIOStatistics(), manifestFiles); + } finally { + entryWriter.close(); + } + final LoadedManifestData loadedManifestData = new LoadedManifestData( + new ArrayList<>(directories.values()), // new array to free up the map + entrySequenceFile, + entryWriter.getCount()); - // collect any stats - maybeAddIOStatistics(getIOStatistics(), manifestFiles); - return new LoadManifestsStage.Result(summaryInfo, null, manifestList, directories); + return new LoadManifestsStage.Result(summaryInfo, loadedManifestData, null); } /** @@ -147,32 +188,55 @@ private void processOneManifest(FileStatus status) TaskManifest m = fetchTaskManifest(status); progress(); - // update the manifest list in a synchronized block. + // update the directories + coalesceDirectories(m); - 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); - } - if (pruneManifests) { + // queue those files. + entryWriter.enqueue(m.getFilesToCommit()); + + // add to the summary. + summaryInfo.add(m); + + // if manifests are cached, clear extra data + // and then save. + if (cacheManifests) { m.setIOStatistics(null); m.getExtraData().clear(); + // update the manifest list in a synchronized block. + synchronized (manifests) { + manifests.add(m); + } } + + } /** * Coalesce all directories and clear the entry in the manifest. - * @param manifest 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 */ - private void coalesceDirectories(TaskManifest manifest) { - final List destDirectories = manifest.getDestDirectories(); - synchronized (directories) { - destDirectories.forEach(entry -> { - directories.putIfAbsent(entry.getDir(), entry); - }); + private void coalesceDirectories(final TaskManifest manifest) { + + // build a list of dirs to create. + // this scans the map + final List 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); + }); + } } - manifest.getDestDirectories().clear(); + } /** @@ -196,9 +260,13 @@ private TaskManifest fetchTaskManifest(FileStatus status) final long size = manifest.getTotalFileSize(); LOG.info("{}: Task Attempt {} file {}: File count: {}; data 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; } @@ -209,16 +277,29 @@ public static final class Arguments { /** * File where the listing has been saved. */ - private final File renameListFile; + private final File entrySequenceFile; /** * build a list of manifests and return them? */ private final boolean cacheManifests; - public Arguments(final File renameListFile, final boolean cacheManifests) { - this.renameListFile = renameListFile; + /** + * Capacity for queue between manifest loader and the writers. + */ + private final int queueCapacity; + + public Arguments(final File entrySequenceFile, + final boolean cacheManifests, + final int queueCapacity) { + this.entrySequenceFile = entrySequenceFile; this.cacheManifests = cacheManifests; + this.queueCapacity = queueCapacity; + } + + private Path getEntrySequenceFile() { + return new Path(entrySequenceFile.toURI()); + } } @@ -228,27 +309,22 @@ public Arguments(final File renameListFile, final boolean cacheManifests) { public static final class Result { private final SummaryInfo summary; - /** - * File where the listing has been saved. - */ - private final File renameListFile; - /** * manifest list, non-null only if cacheManifests is true. */ private final List manifests; /** - * Map of directories. + * Output of this stage to pass on to the subsequence stages. */ - private final Map directories; + private final LoadedManifestData loadedManifestData; public Result(SummaryInfo summary, - final File renameListFile, List manifests, final Map directories) { + final LoadedManifestData loadedManifestData, + final List manifests) { this.summary = summary; - this.renameListFile = renameListFile; this.manifests = manifests; - this.directories = directories; + this.loadedManifestData = loadedManifestData; } public SummaryInfo getSummary() { @@ -259,6 +335,9 @@ public List getManifests() { return manifests; } + public LoadedManifestData getLoadedManifestData() { + return loadedManifestData; + } } /** @@ -269,7 +348,12 @@ public static final class SummaryInfo implements IOStatisticsSource { /** * Aggregate IOStatistics. */ - private IOStatisticsSnapshot iostatistics = snapshotIOStatistics(); + private final IOStatisticsSnapshot iostatistics = snapshotIOStatistics(); + + /** + * Task IDs. + */ + private final List taskIDs = new ArrayList<>(); /** * How many manifests were loaded. @@ -319,16 +403,21 @@ public long getManifestCount() { return manifestCount; } + public List getTaskIDs() { + return taskIDs; + } + /** - * Add all statistics. + * Add all statistics; synchronized. * @param manifest manifest to add. */ - public void add(TaskManifest manifest) { + public synchronized void add(TaskManifest manifest) { manifestCount++; iostatistics.aggregate(manifest.getIOStatistics()); fileCount += manifest.getFilesToCommit().size(); directoryCount += manifest.getDestDirectories().size(); totalFileSize += manifest.getTotalFileSize(); + taskIDs.add(manifest.getTaskID()); } /** diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/StageConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/StageConfig.java index d5bbba5b761e3..59d9c7faae626 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/StageConfig.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/StageConfig.java @@ -18,6 +18,7 @@ package org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; @@ -152,6 +153,13 @@ public class StageConfig { */ private String name = ""; + /** + * Configuration used where needed. + * Default value is a configuration with the normal constructor; + * jobs should override this with what was passed down.j + */ + private Configuration conf = new Configuration(); + public StageConfig() { } @@ -405,6 +413,24 @@ public String getName() { return name; } + /** + * Set configuration. + * @param value new value + * @return the builder + */ + public StageConfig withConfiguration(Configuration value) { + conf = value; + return this; + } + + /** + * Get configuration. + * @return the configuration + */ + public Configuration getConf() { + return conf; + } + /** * Handler for stage entry events. * @return the handler. diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/AbstractManifestCommitterTest.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/AbstractManifestCommitterTest.java index bcd9d34e263de..1a1e44bae4777 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/AbstractManifestCommitterTest.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/AbstractManifestCommitterTest.java @@ -787,6 +787,7 @@ protected StageConfig createStageConfig( jobId, jobAttemptNumber); StageConfig config = new StageConfig(); config + .withConfiguration(getConfiguration()) .withIOProcessors(getSubmitter()) .withIOStatistics(getStageStatistics()) .withJobId(jobId) diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestCreateOutputDirectoriesStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestCreateOutputDirectoriesStage.java index 4b93ce937f80c..c471ef11a88d4 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestCreateOutputDirectoriesStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestCreateOutputDirectoriesStage.java @@ -32,11 +32,9 @@ 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.EntryStatus; -import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.CreateOutputDirectoriesStage; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.SetupJobStage; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.StageConfig; -import org.apache.hadoop.util.Lists; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.lookupCounterStatistic; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; @@ -103,14 +101,14 @@ public void testPrepareSomeDirs() throws Throwable { final long initialFileStatusCount = lookupCounterStatistic(iostats, OP_GET_FILE_STATUS); final int dirCount = 8; + + // add duplicate entries to the list even though in the current iteration + // that couldn't happen. final List dirs = subpaths(destDir, dirCount); final List dirEntries = dirEntries(dirs, 1, EntryStatus.not_found); + dirEntries.addAll(dirEntries(dirs, 1, EntryStatus.not_found)); - // two manifests with duplicate entries - final List manifests = Lists.newArrayList( - manifestWithDirsToCreate(dirEntries), - manifestWithDirsToCreate(dirEntries)); - final CreateOutputDirectoriesStage.Result result = mkdirStage.apply(manifests); + final CreateOutputDirectoriesStage.Result result = mkdirStage.apply(dirEntries); Assertions.assertThat(result.getCreatedDirectories()) .describedAs("output of %s", mkdirStage) .containsExactlyInAnyOrderElementsOf(dirs); @@ -125,8 +123,7 @@ public void testPrepareSomeDirs() throws Throwable { final CreateOutputDirectoriesStage s2 = new CreateOutputDirectoriesStage(stageConfig); final CreateOutputDirectoriesStage.Result r2 = s2.apply( - Lists.newArrayList( - manifestWithDirsToCreate(dirEntries(dirs, 1, EntryStatus.dir)))); + dirEntries(dirs, 1, EntryStatus.dir)); // no directories are now created. Assertions.assertThat(r2.getCreatedDirectories()) @@ -157,19 +154,6 @@ protected List dirEntries(Collection paths, .collect(Collectors.toList()); } - /** - * Create a manifest with the list of directory entries added. - * Job commit requires the entries to have been probed for, and - * for the entire tree under the dest path to be included. - * @param dirEntries list of directory entries. - * @return the manifest. - */ - protected TaskManifest manifestWithDirsToCreate(List dirEntries) { - final TaskManifest taskManifest = new TaskManifest(); - taskManifest.getDestDirectories().addAll(dirEntries); - return taskManifest; - } - /** * Assert the directory map status of a path. * @param result stage result @@ -241,12 +225,9 @@ public void testPrepareDirtyTree() throws Throwable { parentIsDir.setStatus(EntryStatus.dir); leafIsFile.setStatus(EntryStatus.file); - final List manifests = Lists.newArrayList( - manifestWithDirsToCreate(directories)); - // first attempt will succeed. final CreateOutputDirectoriesStage.Result result = - mkdirStage.apply(manifests); + mkdirStage.apply(directories); LOG.info("Job Statistics\n{}", ioStatisticsToPrettyString(iostats)); @@ -270,7 +251,7 @@ public void testPrepareDirtyTree() throws Throwable { // attempt will fail because one of the entries marked as // a file to delete is now a non-empty directory LOG.info("Executing failing attempt to create the directories"); - intercept(IOException.class, () -> attempt2.apply(manifests)); + intercept(IOException.class, () -> attempt2.apply(directories)); verifyStatisticCounterValue(iostats, OP_PREPARE_DIR_ANCESTORS + SUFFIX_FAILURES, 1); verifyStatisticCounterValue(iostats, OP_DELETE + SUFFIX_FAILURES, 1); @@ -281,14 +262,12 @@ public void testPrepareDirtyTree() throws Throwable { directories3.addAll(dirEntries(level2, 2, EntryStatus.dir)); directories3.addAll(dirEntries(level3, 3, EntryStatus.dir)); - final List manifests3 = Lists.newArrayList( - manifestWithDirsToCreate(directories3)); CreateOutputDirectoriesStage attempt3 = new CreateOutputDirectoriesStage( createStageConfigForJob(JOB1, destDir) .withDeleteTargetPaths(true)); final CreateOutputDirectoriesStage.Result r3 = - attempt3.apply(manifests3); + attempt3.apply(directories3); assertDirMapStatus(r3, leafIsFile.getDestPath(), CreateOutputDirectoriesStage.DirMapState.dirFoundInStore); Assertions.assertThat(r3.getCreatedDirectories()) diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java index 4996c2031fa0b..c904b937803bc 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java @@ -37,6 +37,7 @@ import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.InternalConstants; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.OutputValidationException; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.AbortTaskStage; @@ -64,6 +65,7 @@ import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterTestSupport.validateGeneratedFiles; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DiagnosticKeys.PRINCIPAL; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DiagnosticKeys.STAGE; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.InternalConstants.ENTRY_WRITER_QUEUE_CAPACITY; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.manifestPathForTask; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.CleanupJobStage.DISABLED; import static org.apache.hadoop.security.UserGroupInformation.getCurrentUser; @@ -443,8 +445,11 @@ public void test_0340_setupThenAbortTask11() throws Throwable { @Test public void test_0400_loadManifests() throws Throwable { describe("Load all manifests; committed must be TA01 and TA10"); + File entryFile = File.createTempFile("entry", ".seq"); + LoadManifestsStage.Arguments args = new LoadManifestsStage.Arguments( + entryFile, false, InternalConstants.ENTRY_WRITER_QUEUE_CAPACITY); LoadManifestsStage.Result result - = new LoadManifestsStage(getJobStageConfig()).apply(true); + = new LoadManifestsStage(getJobStageConfig()).apply(args); String summary = result.getSummary().toString(); LOG.info("Manifest summary {}", summary); List manifests = result.getManifests(); @@ -484,7 +489,10 @@ public void test_0420_validateJob() throws Throwable { // load manifests stage will load all the task manifests again List manifests = new LoadManifestsStage(getJobStageConfig()) .apply(new LoadManifestsStage.Arguments( - File.createTempFile("manifest", ".list"), true)).getManifests(); + File.createTempFile("manifest", ".list"), + true, + ENTRY_WRITER_QUEUE_CAPACITY)) + .getManifests(); // Now verify their files exist, returning the list of renamed files. List committedFiles = new ValidateRenamedFilesStage(getJobStageConfig()) .apply(manifests) diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestLoadManifestsStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestLoadManifestsStage.java index bf2b466310662..9c8ab98211dfe 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestLoadManifestsStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestLoadManifestsStage.java @@ -18,11 +18,9 @@ package org.apache.hadoop.mapreduce.lib.output.committer.manifest; -import java.util.HashMap; +import java.io.File; import java.util.List; -import java.util.Map; import java.util.Set; -import java.util.stream.Collectors; import org.assertj.core.api.Assertions; import org.junit.Test; @@ -31,10 +29,9 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; -import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestPrinter; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData; -import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.InternalConstants; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.CleanupJobStage; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.CreateOutputDirectoriesStage; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.LoadManifestsStage; @@ -63,6 +60,8 @@ public class TestLoadManifestsStage extends AbstractManifestCommitterTest { private int taskAttemptCount; + private File entryFile; + /** * How many task attempts to make? * Override point. @@ -81,9 +80,17 @@ public void setup() throws Exception { .isGreaterThan(0); } + @Override + public void teardown() throws Exception { + if (entryFile != null) { + entryFile.delete(); + } + super.teardown(); + } + public long heapSize() { - return Runtime.getRuntime().totalMemory() - Runtime.getRuntime().freeMemory(); - } + return Runtime.getRuntime().totalMemory() - Runtime.getRuntime().freeMemory(); + } /** * Build a large number of manifests, but without the real files @@ -123,10 +130,11 @@ public void testSaveThenLoadManyManifests() throws Throwable { // Load in the manifests LoadManifestsStage stage = new LoadManifestsStage( stageConfig); - - LoadManifestsStage.Result result = stage.apply(true); + entryFile = File.createTempFile("entry", ".seq"); + LoadManifestsStage.Arguments args = new LoadManifestsStage.Arguments( + entryFile, false, InternalConstants.ENTRY_WRITER_QUEUE_CAPACITY); + LoadManifestsStage.Result result = stage.apply(args); LoadManifestsStage.SummaryInfo summary = result.getSummary(); - List loadedManifests = result.getManifests(); LOG.info("\nJob statistics after loading {}", ioStatisticsToPrettyString(getStageStatistics())); @@ -134,7 +142,6 @@ public void testSaveThenLoadManyManifests() throws Throwable { addHeapInformation(heapInfo, "load.manifests"); - Assertions.assertThat(summary.getManifestCount()) .describedAs("Manifest count of %s", summary) .isEqualTo(taskAttemptCount); @@ -147,9 +154,7 @@ public void testSaveThenLoadManyManifests() throws Throwable { // now that manifest list. - List manifestTaskIds = loadedManifests.stream() - .map(TaskManifest::getTaskID) - .collect(Collectors.toList()); + List manifestTaskIds = summary.getTaskIDs(); Assertions.assertThat(getTaskIds()) .describedAs("Task IDs of all tasks") .containsExactlyInAnyOrderElementsOf(manifestTaskIds); @@ -157,7 +162,7 @@ public void testSaveThenLoadManyManifests() throws Throwable { // now let's see about aggregating a large set of directories Set createdDirectories = new CreateOutputDirectoriesStage( stageConfig) - .apply(loadedManifests) + .apply(result.getLoadedManifestData().getDirectories()) .getCreatedDirectories(); addHeapInformation(heapInfo, "create.directories"); @@ -187,8 +192,8 @@ public void testSaveThenLoadManyManifests() throws Throwable { success.save(summaryFS, path, true); LOG.info("Saved summary to {}", path); ManifestPrinter showManifest = new ManifestPrinter(); - ManifestSuccessData manifestSuccessData = - showManifest.loadAndPrintManifest(summaryFS, path); + ManifestSuccessData manifestSuccessData = + showManifest.loadAndPrintManifest(summaryFS, path); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/TestEntryFileIO.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/TestEntryFileIO.java new file mode 100644 index 0000000000000..384fe7dd97211 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/TestEntryFileIO.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl; + +import java.io.File; +import java.util.ArrayList; +import java.util.List; + +import org.assertj.core.api.Assertions; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry; +import org.apache.hadoop.test.AbstractHadoopTestBase; +import org.apache.hadoop.util.functional.RemoteIterators; + +/** + * Test {@link EntryFileIO}. + */ +public class TestEntryFileIO extends AbstractHadoopTestBase { + + private EntryFileIO entryFileIO; + + private File entryFile; + + @Before + public void setup() throws Exception { + entryFileIO = new EntryFileIO(new Configuration()); + } + /** + * Teardown. + * @throws Exception on any failure + */ + @After + public void teardown() throws Exception { + Thread.currentThread().setName("teardown"); + if (entryFile != null) { + entryFile.delete(); + } + } + + + @Test + public void testCreateWriteReadFile() throws Throwable { + entryFile = File.createTempFile("entry", ".seq"); + final FileEntry source = new FileEntry("source", "dest", 100, "etag"); + SequenceFile.Writer writer = entryFileIO.createWriter(entryFile); + writer.append(NullWritable.get(), source); + writer.flush(); + writer.close(); + Assertions.assertThat(entryFile.length()) + .describedAs("Length of file %s", entryFile) + .isGreaterThan(0); + + FileEntry readBack = new FileEntry(); + try (SequenceFile.Reader reader = entryFileIO.createReader(entryFile)) { + reader.next(NullWritable.get(), readBack); + } + Assertions.assertThat(readBack) + .describedAs("entry read back from sequence file") + .isEqualTo(source); + + final RemoteIterator it = + entryFileIO.iterateOver(entryFileIO.createReader(entryFile)); + List entries = new ArrayList<>(); + RemoteIterators.foreach(it, entries::add); + Assertions.assertThat(entries) + .hasSize(1) + .element(0) + .isEqualTo(source); + + } + +} From f95a926843148626abe5b24e81be7ea7abf099cf Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 5 Apr 2023 15:52:46 +0100 Subject: [PATCH 04/18] MAPREDUCE-7435. oom: switch to sequence file for storage of the files. interim commit Change-Id: I80bb4e72c1029baad8fb87d8c9287b08c0b000f4 --- .../committer/manifest/impl/EntryFileIO.java | 73 +++++++++++++++---- .../manifest/stages/CommitJobStage.java | 6 +- .../manifest/stages/LoadManifestsStage.java | 32 +++++--- .../manifest/stages/RenameFilesStage.java | 47 +++++++----- 4 files changed, 112 insertions(+), 46 deletions(-) diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java index efa561b06405a..7d896f20861b6 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java @@ -25,10 +25,12 @@ import java.util.ArrayList; import java.util.List; import java.util.NoSuchElementException; -import java.util.Queue; import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import org.slf4j.Logger; @@ -138,7 +140,7 @@ public final class EntryWriter implements Closeable { private final SequenceFile.Writer writer; - private final Queue> queue; + private final BlockingQueue> queue; /** * stop flag. @@ -158,7 +160,7 @@ public final class EntryWriter implements Closeable { private Future future; /** - * count of files opened; only updated in one thread + * count of file entries saved; only updated in one thread * so volatile. */ private volatile int count; @@ -218,6 +220,10 @@ private void start() { * @return whether the queue worked. */ public boolean enqueue(List entries) { + if (entries.isEmpty()) { + // exit fast, but return true. + return true; + } if (active.get()) { queue.add(entries); return false; @@ -234,8 +240,19 @@ public boolean enqueue(List entries) { */ private int processor() { int count = 0; - while (!stop.get()) { - queue.poll().forEach(this::append); + try { + while (!stop.get()) { + final List entries = queue.take(); + if (entries.isEmpty()) { + // exit list reached. + stop.set(true); + } else { + entries.forEach(this::append); + } + } + } catch (InterruptedException e) { + // assume we are stopped here + stop.set(true); } return count; } @@ -257,32 +274,53 @@ private void append(FileEntry entry) { } } + /** + * Close: stop accepting new writes, wait for queued writes to complete + * @throws IOException failure closing that writer, or somehow the future + * raises an IOE which isn't caught for later. + */ + @Override public void close() throws IOException { - if (stop.getAndSet(true)) { + + // declare as inactive. + // this stops queueing more data, but leaves + // the worker thread still polling and writing. + if (!active.getAndSet(false)) { // already stopped return; } LOG.debug("Shutting down writer"); - // signal queue closure by - // clearing the current list - // and queue an empty list - queue.clear(); + // signal queue closure by queue an empty list queue.add(new ArrayList<>()); try { // wait for the op to finish. - final int count = FutureIO.awaitFuture(future); - LOG.debug("Processed {} files", count); + int total = FutureIO.awaitFuture(future, 1, TimeUnit.MINUTES); + LOG.debug("Processed {} files", total); + executor.shutdown(); + } catch (TimeoutException e) { + // trouble. force close + executor.shutdownNow(); // close the stream } finally { writer.close(); } } + + /** + * Raise any IOException caught during execution of the writer thread. + * @throws IOException if one was caught and saved. + */ + public void maybeRaiseWriteException() throws IOException { + if (failure != null) { + throw failure; + } + } } /** * Iterator to retrieve file entries from the sequence file. - * Closeable. + * Closeable; it will close automatically when the last element is read. */ private final class EntryIterator implements RemoteIterator, Closeable { @@ -290,13 +328,18 @@ private final class EntryIterator implements RemoteIterator, Closeabl private FileEntry fetched; + private boolean closed; + private EntryIterator(final SequenceFile.Reader reader) { this.reader = requireNonNull(reader); } @Override public void close() throws IOException { - reader.close(); + if (!closed) { + closed = true; + reader.close(); + } } @Override @@ -311,6 +354,7 @@ private boolean fetchNext() throws IOException { return true; } else { fetched = null; + close(); return false; } } @@ -324,6 +368,7 @@ public FileEntry next() throws IOException { fetched = null; return r; } + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java index c8ead7be4e669..7ecd16e46db77 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java @@ -82,9 +82,9 @@ protected CommitJobStage.Result executeStage( final StageConfig stageConfig = getStageConfig(); LoadManifestsStage.Result result = new LoadManifestsStage(stageConfig).apply( new LoadManifestsStage.Arguments( - File.createTempFile("manifest", ".list"), false, + File.createTempFile("manifest", ".list"), + false, ENTRY_WRITER_QUEUE_CAPACITY)); - List manifests = result.getManifests(); LoadManifestsStage.SummaryInfo summary = result.getSummary(); final LoadedManifestData manifestData = result.getLoadedManifestData(); @@ -112,7 +112,7 @@ protected CommitJobStage.Result executeStage( // and hence all aggregate stats from the tasks. ManifestSuccessData successData; successData = new RenameFilesStage(stageConfig).apply( - Pair.of(manifests, dirStageResults.getCreatedDirectories())); + Pair.of(manifestData, dirStageResults.getCreatedDirectories())); if (LOG.isDebugEnabled()) { LOG.debug("{}: _SUCCESS file summary {}", getName(), successData.toJson()); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java index 8af2a38b6fbb7..71526d1c8a3fc 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java @@ -29,6 +29,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathIOException; @@ -146,6 +147,9 @@ protected LoadManifestsStage.Result executeStage( // close cleanly entryWriter.close(); + // if anything failed, raise it. + entryWriter.maybeRaiseWriteException(); + // collect any stats maybeAddIOStatistics(getIOStatistics(), manifestFiles); } finally { @@ -185,30 +189,35 @@ private void processOneManifest(FileStatus status) throws IOException { updateAuditContext(OP_LOAD_ALL_MANIFESTS); - TaskManifest m = fetchTaskManifest(status); + TaskManifest manifest = fetchTaskManifest(status); progress(); // update the directories - coalesceDirectories(m); + final int created = coalesceDirectories(manifest); + LOG.debug("{}: task {} added {} directories", + getName(), manifest.getTaskID(), created); // queue those files. - entryWriter.enqueue(m.getFilesToCommit()); + final boolean enqueued = entryWriter.enqueue(manifest.getFilesToCommit()); + if (!enqueued) { + LOG.warn("{}: Failed to write manifest for task {}", + getName(), + manifest.getTaskID()); + } // add to the summary. - summaryInfo.add(m); + summaryInfo.add(manifest); // if manifests are cached, clear extra data // and then save. if (cacheManifests) { - m.setIOStatistics(null); - m.getExtraData().clear(); + manifest.setIOStatistics(null); + manifest.getExtraData().clear(); // update the manifest list in a synchronized block. synchronized (manifests) { - manifests.add(m); + manifests.add(manifest); } } - - } /** @@ -217,8 +226,10 @@ private void processOneManifest(FileStatus status) * 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; */ - private void coalesceDirectories(final TaskManifest manifest) { + @VisibleForTesting + int coalesceDirectories(final TaskManifest manifest) { // build a list of dirs to create. // this scans the map @@ -236,6 +247,7 @@ private void coalesceDirectories(final TaskManifest manifest) { }); } } + return toCreate.size(); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/RenameFilesStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/RenameFilesStage.java index e8124c11465e5..066e5343b3681 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/RenameFilesStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/RenameFilesStage.java @@ -18,6 +18,7 @@ package org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages; +import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -29,16 +30,18 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData; -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 static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SUCCESS_MARKER_FILE_LIMIT; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_COMMIT; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_RENAME_FILES; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.createManifestOutcome; -import static org.apache.hadoop.thirdparty.com.google.common.collect.Iterables.concat; /** * This stage renames all the files. @@ -51,7 +54,7 @@ */ public class RenameFilesStage extends AbstractJobOrTaskStage< - Pair, Set>, + Pair>, ManifestSuccessData> { private static final Logger LOG = LoggerFactory.getLogger( @@ -92,37 +95,43 @@ public synchronized long getTotalFileSize() { /** * Rename files in job commit. - * @param taskManifests a list of task manifests containing files. + * @param args tuple of (manifest data, set of created dirs) * @return the job report. * @throws IOException failure */ @Override protected ManifestSuccessData executeStage( - Pair, Set> args) + Pair> args) throws IOException { - final List taskManifests = args.getLeft(); + + final LoadedManifestData manifestData = args.getLeft(); createdDirectories = args.getRight(); + final EntryFileIO entryFileIO = new EntryFileIO(getStageConfig().getConf()); + final SequenceFile.Reader reader = + entryFileIO.createReader(manifestData.getEntrySequenceFile()); + final ManifestSuccessData success = createManifestOutcome(getStageConfig(), OP_STAGE_JOB_COMMIT); - final int manifestCount = taskManifests.size(); - LOG.info("{}: Executing Manifest Job Commit with {} manifests in {}", - getName(), manifestCount, getTaskManifestDir()); + LOG.info("{}: Executing Manifest Job Commit with {} files", + getName(), manifestData.getFileCount()); - // first step is to aggregate the output of all manifests into a single - // list of files to commit. - // Which Guava can do in a zero-copy concatenated iterator + // iterate over the - final Iterable filesToCommit = concat(taskManifests.stream() - .map(TaskManifest::getFilesToCommit) - .collect(Collectors.toList())); + final RemoteIterator entries = entryFileIO.iterateOver(reader); + try { + TaskPool.foreach(entries) + .executeWith(getIOProcessors()) + .stopOnFailure() + .run(this::commitOneFile); + } finally { + // close the input. Automatic on a successful scan, but + // a rename failure will abort the operation. + ((Closeable)entries).close(); - TaskPool.foreach(filesToCommit) - .executeWith(getIOProcessors()) - .stopOnFailure() - .run(this::commitOneFile); + } // synchronized block to keep spotbugs happy. List committed = getFilesCommitted(); From 3046b082e4a6a9d52b6f4cfc11c6720fef780b8c Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 5 Apr 2023 16:44:37 +0100 Subject: [PATCH 05/18] MAPREDUCE-7435. starting to get write/read chain working ...but not tested the job commit yet Change-Id: I0f54ede94e41592558468df1c87f4a39d2461223 --- .../committer/manifest/files/FileEntry.java | 7 ++-- .../committer/manifest/impl/EntryFileIO.java | 41 ++++++++++++++++++- .../manifest/impl/LoadedManifestData.java | 12 ++++-- .../manifest/stages/RenameFilesStage.java | 2 +- .../ManifestCommitterTestSupport.java | 23 +++++++++++ .../manifest/TestRenameStageFailure.java | 26 ++++++++++-- 6 files changed, 97 insertions(+), 14 deletions(-) diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/FileEntry.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/FileEntry.java index ac4498d6d9d8b..877c2843ade53 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/FileEntry.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/FileEntry.java @@ -35,6 +35,7 @@ import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableUtils; +import static java.util.Objects.requireNonNull; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData.marshallPath; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData.unmarshallPath; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData.verify; @@ -196,9 +197,9 @@ public int hashCode() { @Override public void write(final DataOutput out) throws IOException { - Text.writeString(out, source); - Text.writeString(out, dest); - Text.writeString(out, etag); + Text.writeString(out, requireNonNull(source, "null source")); + Text.writeString(out, requireNonNull(dest, "null dest")); + Text.writeString(out, etag != null ? etag : ""); WritableUtils.writeVLong(out, size); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java index 7d896f20861b6..9714e2bf19987 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.util.ArrayList; +import java.util.Collection; import java.util.List; import java.util.NoSuchElementException; import java.util.concurrent.ArrayBlockingQueue; @@ -78,7 +79,7 @@ public EntryFileIO(final Configuration conf) { * @throws IOException fail to open the file */ public SequenceFile.Writer createWriter(File file) throws IOException { - return createWriter(new Path(file.toURI())); + return createWriter(toPath(file)); } public SequenceFile.Writer createWriter(Path path) throws IOException { @@ -96,7 +97,7 @@ public SequenceFile.Writer createWriter(Path path) throws IOException { * @throws IOException failure to open */ public SequenceFile.Reader createReader(File file) throws IOException { - return createReader(new Path(file.toURI())); + return createReader(toPath(file)); } /** @@ -132,6 +133,42 @@ public EntryWriter launchEntryWriter(SequenceFile.Writer writer, int capacity) { return ew; } + /** + * Write a sequence of entries to the writer. + * @param writer writer + * @param entries entries + * @param close close the stream afterwards + * @return number of entries written + * @throws IOException write failure. + */ + public static int write(SequenceFile.Writer writer, + Collection entries, + boolean close) + throws IOException { + try { + for (FileEntry entry: entries) { + writer.append(NullWritable.get(), entry); + } + writer.flush(); + } finally { + if (close) { + writer.close(); + } + } + return entries.size(); + } + + + /** + * Given a file, create a Path. + * @param file file + * @return path to the file + */ + public static Path toPath(final File file) { + return new Path(file.toURI()); + } + + /** * Writer takes a list of entries at a time; queues for writing. * A special diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/LoadedManifestData.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/LoadedManifestData.java index 1aed6d1fcbd5c..690eff761811c 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/LoadedManifestData.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/LoadedManifestData.java @@ -18,6 +18,7 @@ package org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl; +import java.io.File; import java.util.Collection; import org.apache.hadoop.fs.Path; @@ -41,7 +42,7 @@ public final class LoadedManifestData { * files to rename. * This will be a sequence file of long -> FileEntry */ - private final Path entrySequenceFile; + private final Path entrySequenceData; /** * How many files will be renamed. @@ -54,7 +55,7 @@ public LoadedManifestData( final int fileCount) { this.directories = directories; this.fileCount = fileCount; - this.entrySequenceFile = entrySequenceFile; + this.entrySequenceData = entrySequenceFile; } public Collection getDirectories() { @@ -65,8 +66,11 @@ public int getFileCount() { return fileCount; } - public Path getEntrySequenceFile() { - return entrySequenceFile; + public Path getEntrySequenceData() { + return entrySequenceData; } + public File getEntrySequenceFile() { + return new File(entrySequenceData.toUri()); + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/RenameFilesStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/RenameFilesStage.java index 066e5343b3681..2bdf7f165a331 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/RenameFilesStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/RenameFilesStage.java @@ -109,7 +109,7 @@ protected ManifestSuccessData executeStage( createdDirectories = args.getRight(); final EntryFileIO entryFileIO = new EntryFileIO(getStageConfig().getConf()); final SequenceFile.Reader reader = - entryFileIO.createReader(manifestData.getEntrySequenceFile()); + entryFileIO.createReader(manifestData.getEntrySequenceData()); final ManifestSuccessData success = createManifestOutcome(getStageConfig(), diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterTestSupport.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterTestSupport.java index 31abcb63f5368..20bd6e93baa15 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterTestSupport.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterTestSupport.java @@ -37,6 +37,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.mapreduce.JobID; import org.apache.hadoop.mapreduce.RecordWriter; import org.apache.hadoop.mapreduce.TaskAttemptContext; @@ -47,11 +48,15 @@ import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestPrinter; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData; +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.RemoteIterators; import static org.apache.commons.lang3.StringUtils.isNotEmpty; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.MANIFEST_COMMITTER_CLASSNAME; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SUCCESS_MARKER; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.EntryFileIO.toPath; import static org.assertj.core.api.Assertions.assertThat; /** @@ -295,6 +300,24 @@ static void assertDirEntryMatch( .isEqualTo(type); } + /** + * Save a manifest to an entry file; returning the loaded manifest data. + * Caller MUST clean up the temp file. + * @param entryFileIO IO class + * @param manifest manifest to process. + * @return info about the load + * @throws IOException write failure + */ + public static LoadedManifestData saveManifest(EntryFileIO entryFileIO, TaskManifest manifest) + throws IOException { + final File tempFile = File.createTempFile("entries", ".seq"); + final SequenceFile.Writer writer = entryFileIO.createWriter(tempFile); + return new LoadedManifestData( + manifest.getDestDirectories(), + toPath(tempFile), + EntryFileIO.write(writer, manifest.getFilesToCommit(), true)); + } + /** * Closeable which can be used to safely close writers in * a try-with-resources block.. diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestRenameStageFailure.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestRenameStageFailure.java index 11d2beda5d015..9a482489d5085 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestRenameStageFailure.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestRenameStageFailure.java @@ -39,6 +39,8 @@ import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry; 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.mapreduce.lib.output.committer.manifest.impl.ManifestStoreOperations; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.UnreliableManifestStoreOperations; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.RenameFilesStage; @@ -49,6 +51,7 @@ import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter; import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_COMMIT_FILE_RENAME; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterTestSupport.saveManifest; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.getEtag; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.UnreliableManifestStoreOperations.SIMULATED_FAILURE; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.AbstractJobOrTaskStage.FAILED_TO_RENAME_PREFIX; @@ -82,6 +85,8 @@ public class TestRenameStageFailure extends AbstractManifestCommitterTest { /** resilient commit expected? */ private boolean resilientCommit; + private EntryFileIO entryFileIO; + protected boolean isResilientCommit() { return resilientCommit; } @@ -109,6 +114,7 @@ public void setup() throws Exception { = new UnreliableManifestStoreOperations(wrappedOperations); setStoreOperations(failures); resilientCommit = wrappedOperations.storeSupportsResilientCommit(); + entryFileIO = new EntryFileIO(getConfiguration()); } /** @@ -232,9 +238,15 @@ public void testDeleteTargetPaths() throws Throwable { LOG.info("Exception raised: {}", ex.toString()); } + final LoadedManifestData manifestData = saveManifest(entryFileIO, manifest); + // delete target paths and it works - new RenameFilesStage(stageConfig.withDeleteTargetPaths(true)) - .apply(Pair.of(manifests, Collections.emptySet())); + try { + new RenameFilesStage(stageConfig.withDeleteTargetPaths(true)) + .apply(Pair.of(manifestData, Collections.emptySet())); + } finally { + manifestData.getEntrySequenceFile().delete(); + } // and the new data made it over verifyFileContents(fs, dest, sourceData); @@ -348,9 +360,15 @@ private E expectRenameFailure( IOStatisticsStore iostatistics = stage.getIOStatistics(); long failures0 = iostatistics.counters().get(RENAME_FAILURES); + final LoadedManifestData manifestData = saveManifest(entryFileIO, manifest); // rename MUST raise an exception. - E ex = intercept(exceptionClass, errorText, () -> - stage.apply(Pair.of(manifests, Collections.emptySet()))); + E ex; + try { + ex = intercept(exceptionClass, errorText, () -> + stage.apply(Pair.of(manifestData, Collections.emptySet()))); + } finally { + manifestData.getEntrySequenceFile().delete(); + } LOG.info("Statistics {}", ioStatisticsToPrettyString(iostatistics)); // the IOStatistics record the rename as a failure. From 3182c97d4dd638eaf7b9b1197cf1543b3a918b07 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 5 Apr 2023 16:49:15 +0100 Subject: [PATCH 06/18] MAPREDUCE-7435. starting to get write/read chain working ...but not tested the job commit yet Change-Id: I4d50636542673a3f25a7ab363df1b1bd221216ae --- .../committer/manifest/stages/LoadManifestsStage.java | 6 ++++-- .../committer/manifest/TestJobThroughManifestCommitter.java | 2 +- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java index 71526d1c8a3fc..95b33c602511c 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java @@ -127,6 +127,7 @@ protected LoadManifestsStage.Result executeStage( entryWriter = entryFileIO.launchEntryWriter( entryFileIO.createWriter(entrySequenceFile), arguments.queueCapacity); + List manifestList; try { // sync fs before the listj @@ -137,7 +138,8 @@ protected LoadManifestsStage.Result executeStage( final RemoteIterator manifestFiles = haltableRemoteIterator(listManifests(), () -> entryWriter.isActive()); - final List manifestList = loadAllManifests(manifestFiles); + manifestList = loadAllManifests(manifestFiles); + LOG.info("{}: Summary of {} manifests loaded in {}: {}", getName(), manifestList.size(), @@ -160,7 +162,7 @@ protected LoadManifestsStage.Result executeStage( entrySequenceFile, entryWriter.getCount()); - return new LoadManifestsStage.Result(summaryInfo, loadedManifestData, null); + return new LoadManifestsStage.Result(summaryInfo, loadedManifestData, manifestList); } /** diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java index c904b937803bc..7de38557cdf0c 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java @@ -447,7 +447,7 @@ public void test_0400_loadManifests() throws Throwable { describe("Load all manifests; committed must be TA01 and TA10"); File entryFile = File.createTempFile("entry", ".seq"); LoadManifestsStage.Arguments args = new LoadManifestsStage.Arguments( - entryFile, false, InternalConstants.ENTRY_WRITER_QUEUE_CAPACITY); + entryFile, true, InternalConstants.ENTRY_WRITER_QUEUE_CAPACITY); LoadManifestsStage.Result result = new LoadManifestsStage(getJobStageConfig()).apply(args); String summary = result.getSummary().toString(); From f18e9daa67f494f81610a37b80b47618d4c7effd Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 6 Apr 2023 18:48:25 +0100 Subject: [PATCH 07/18] MAPREDUCE-7435. Async queue/write working * TestEntryFileIO extended for ths * ABFS terasort test happy! Change-Id: I068861973114d9947f3d22eaf32a6ee3b7ca8fa2 TODO: fault injection on the writes --- .../committer/manifest/impl/EntryFileIO.java | 94 ++++++++--- .../manifest/impl/TestEntryFileIO.java | 152 ++++++++++++++++-- 2 files changed, 212 insertions(+), 34 deletions(-) diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java index 9714e2bf19987..aee6a314dcbe1 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java @@ -33,10 +33,12 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; @@ -141,12 +143,12 @@ public EntryWriter launchEntryWriter(SequenceFile.Writer writer, int capacity) { * @return number of entries written * @throws IOException write failure. */ - public static int write(SequenceFile.Writer writer, + public static int write(SequenceFile.Writer writer, Collection entries, boolean close) throws IOException { try { - for (FileEntry entry: entries) { + for (FileEntry entry : entries) { writer.append(NullWritable.get(), entry); } writer.flush(); @@ -200,7 +202,7 @@ public final class EntryWriter implements Closeable { * count of file entries saved; only updated in one thread * so volatile. */ - private volatile int count; + private final AtomicInteger count = new AtomicInteger(); /** * any failure. @@ -230,7 +232,7 @@ public boolean isActive() { * @return the count */ public int getCount() { - return count; + return count.get(); } /** @@ -249,6 +251,7 @@ private void start() { active.set(true); executor = HadoopExecutors.newSingleThreadExecutor(); future = executor.submit(this::processor); + LOG.debug("Started entry writer {}", this); } /** @@ -258,12 +261,19 @@ private void start() { */ public boolean enqueue(List entries) { if (entries.isEmpty()) { + LOG.debug("ignoring enqueue of empty list"); // exit fast, but return true. return true; } if (active.get()) { - queue.add(entries); - return false; + try { + queue.put(entries); + LOG.debug("Queued {}", entries.size()); + return true; + } catch (InterruptedException e) { + Thread.interrupted(); + return false; + } } else { LOG.debug("Queue inactive; discarding {} entries", entries.size()); return false; @@ -276,22 +286,27 @@ public boolean enqueue(List entries) { * @throws UncheckedIOException on write failure */ private int processor() { - int count = 0; try { while (!stop.get()) { final List entries = queue.take(); if (entries.isEmpty()) { // exit list reached. + LOG.debug("List termination initiated"); stop.set(true); } else { - entries.forEach(this::append); + LOG.debug("Adding block of {} entries", entries.size()); + for (FileEntry entry : entries) { + append(entry); + } } } } catch (InterruptedException e) { // assume we are stopped here + LOG.debug("interrupted", e); + active.set(false); stop.set(true); } - return count; + return count.get(); } /** @@ -300,14 +315,15 @@ private int processor() { * @throws UncheckedIOException on write failure */ private void append(FileEntry entry) { - if (failure != null) { - try { - writer.append(NullWritable.get(), entry); - count++; - } catch (IOException e) { - failure = e; - throw new UncheckedIOException(e); - } + try { + writer.append(NullWritable.get(), entry); + + final int c = count.incrementAndGet(); + LOG.trace("Added entry #{}: {}", c, entry); + } catch (IOException e) { + LOG.debug("Write failure", e); + failure = e; + throw new UncheckedIOException(e); } } @@ -329,7 +345,11 @@ public void close() throws IOException { } LOG.debug("Shutting down writer"); // signal queue closure by queue an empty list - queue.add(new ArrayList<>()); + try { + queue.put(new ArrayList<>()); + } catch (InterruptedException e) { + Thread.interrupted(); + } try { // wait for the op to finish. int total = FutureIO.awaitFuture(future, 1, TimeUnit.MINUTES); @@ -353,13 +373,26 @@ public void maybeRaiseWriteException() throws IOException { throw failure; } } + + @Override + public String toString() { + return "EntryWriter{" + + "stop=" + stop.get() + + ", active=" + active.get() + + ", count=" + count.get() + + ", queue depth=" + queue.size() + + ", failure=" + failure + + '}'; + } } /** * Iterator to retrieve file entries from the sequence file. * Closeable; it will close automatically when the last element is read. + * No thread safety. */ - private final class EntryIterator implements RemoteIterator, Closeable { + @VisibleForTesting + final class EntryIterator implements RemoteIterator, Closeable { private final SequenceFile.Reader reader; @@ -367,6 +400,8 @@ private final class EntryIterator implements RemoteIterator, Closeabl private boolean closed; + private int count; + private EntryIterator(final SequenceFile.Reader reader) { this.reader = requireNonNull(reader); } @@ -379,6 +414,15 @@ public void close() throws IOException { } } + @Override + public String toString() { + return "EntryIterator{" + + "closed=" + closed + + ", count=" + count + + ", fetched=" + fetched + + '}'; + } + @Override public boolean hasNext() throws IOException { return fetched != null || fetchNext(); @@ -388,6 +432,7 @@ private boolean fetchNext() throws IOException { FileEntry readBack = new FileEntry(); if (reader.next(NullWritable.get(), readBack)) { fetched = readBack; + count++; return true; } else { fetched = null; @@ -406,6 +451,17 @@ public FileEntry next() throws IOException { return r; } + /** + * Is the stream closed. + * @return true if closed. + */ + public boolean isClosed() { + return closed; + } + + int getCount() { + return count; + } } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/TestEntryFileIO.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/TestEntryFileIO.java index 384fe7dd97211..4fd9fc0d8f6f8 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/TestEntryFileIO.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/TestEntryFileIO.java @@ -19,8 +19,10 @@ package org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl; import java.io.File; +import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; import org.assertj.core.api.Assertions; import org.junit.After; @@ -33,13 +35,19 @@ import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry; import org.apache.hadoop.test.AbstractHadoopTestBase; -import org.apache.hadoop.util.functional.RemoteIterators; + +import static org.apache.hadoop.util.functional.RemoteIterators.foreach; /** * Test {@link EntryFileIO}. */ public class TestEntryFileIO extends AbstractHadoopTestBase { + public static final FileEntry ENTRY = new FileEntry("source", "dest", 100, "etag"); + + /** + * Entry file instance. + */ private EntryFileIO entryFileIO; private File entryFile; @@ -47,7 +55,9 @@ public class TestEntryFileIO extends AbstractHadoopTestBase { @Before public void setup() throws Exception { entryFileIO = new EntryFileIO(new Configuration()); + createEntryFile(); } + /** * Teardown. * @throws Exception on any failure @@ -55,40 +65,152 @@ public void setup() throws Exception { @After public void teardown() throws Exception { Thread.currentThread().setName("teardown"); - if (entryFile != null) { - entryFile.delete(); + if (getEntryFile() != null) { + getEntryFile().delete(); } } + private void createEntryFile() throws IOException { + setEntryFile(File.createTempFile("entry", ".seq")); + } + + /** + * reference to any temp file created. + */ + private File getEntryFile() { + return entryFile; + } + + private void setEntryFile(File entryFile) { + this.entryFile = entryFile; + } + + + /** + * Create a file with one entry + */ @Test - public void testCreateWriteReadFile() throws Throwable { - entryFile = File.createTempFile("entry", ".seq"); - final FileEntry source = new FileEntry("source", "dest", 100, "etag"); - SequenceFile.Writer writer = entryFileIO.createWriter(entryFile); + public void testCreateWriteReadFileOneEntry() throws Throwable { + + final FileEntry source = ENTRY; + + // do an explicit close to help isolate any failure. + SequenceFile.Writer writer = createWriter(); writer.append(NullWritable.get(), source); writer.flush(); writer.close(); - Assertions.assertThat(entryFile.length()) - .describedAs("Length of file %s", entryFile) - .isGreaterThan(0); FileEntry readBack = new FileEntry(); - try (SequenceFile.Reader reader = entryFileIO.createReader(entryFile)) { + try (SequenceFile.Reader reader = readEntryFile()) { reader.next(NullWritable.get(), readBack); } Assertions.assertThat(readBack) .describedAs("entry read back from sequence file") .isEqualTo(source); + // now use the iterator to access it. final RemoteIterator it = - entryFileIO.iterateOver(entryFileIO.createReader(entryFile)); - List entries = new ArrayList<>(); - RemoteIterators.foreach(it, entries::add); - Assertions.assertThat(entries) + iterateOverEntryFile(); + List files = new ArrayList<>(); + foreach(it, files::add); + Assertions.assertThat(files) .hasSize(1) .element(0) .isEqualTo(source); + final EntryFileIO.EntryIterator et = (EntryFileIO.EntryIterator) it; + Assertions.assertThat(et) + .describedAs("entry iterator %s", et) + .matches(p -> p.isClosed()) + .extracting(p -> p.getCount()) + .isEqualTo(1); + } + + private SequenceFile.Writer createWriter() throws IOException { + return entryFileIO.createWriter(getEntryFile()); + } + + private RemoteIterator iterateOverEntryFile() throws IOException { + return entryFileIO.iterateOver(readEntryFile()); + } + + private SequenceFile.Reader readEntryFile() throws IOException { + assertEntryFileNonEmpty(); + + return entryFileIO.createReader(getEntryFile()); + } + + /** + * Create a file with one entry + */ + @Test + public void testCreateEmptyFile() throws Throwable { + + final File file = getEntryFile(); + + entryFileIO.createWriter(file).close(); + + // now use the iterator to access it. + List files = new ArrayList<>(); + Assertions.assertThat(foreach(iterateOverEntryFile(), files::add)) + .isEqualTo(0); + } + + private void assertEntryFileNonEmpty() { + Assertions.assertThat(getEntryFile().length()) + .describedAs("Length of file %s", getEntryFile()) + .isGreaterThan(0); + } + + /** + * Generate lots of data and write. + */ + @Test + public void testLargeStreamingWrite() throws Throwable { + + // list of 100 entries at a time + int listSize = 100; + // and the number of block writes + int writes = 100; + List list = new ArrayList<>(listSize); + for (int i = 0; i < listSize; i++) { + list.add(new FileEntry("source" + i, "dest" + i, i, "etag-" + i)); + } + // just for debugging/regression testing + Assertions.assertThat(list).hasSize(listSize); + int total = listSize * writes; + + try (EntryFileIO.EntryWriter out = entryFileIO.launchEntryWriter(createWriter(), 2)) { + Assertions.assertThat(out.isActive()) + .describedAs("out.isActive in ()", out) + .isTrue(); + for (int i = 0; i < writes; i++) { + Assertions.assertThat(out.enqueue(list)) + .describedAs("enqueue of list") + .isTrue(); + } + out.close(); + out.maybeRaiseWriteException(); + Assertions.assertThat(out.isActive()) + .describedAs("out.isActive in ()", out) + .isFalse(); + + Assertions.assertThat(out.getCount()) + .describedAs("total elements written") + .isEqualTo(total); + } + AtomicInteger count = new AtomicInteger(); + foreach(iterateOverEntryFile(), e -> { + final int elt = count.getAndIncrement(); + final int index = elt % listSize; + Assertions.assertThat(e) + .describedAs("element %d in file mapping to index %d", elt, index) + .isEqualTo(list.get(index)); + }); + Assertions.assertThat(count.get()) + .describedAs("total elements read") + .isEqualTo(total); + } From 30d90e07cbd5915b14e8caf6a10277db9cc6d542 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 17 Apr 2023 19:47:48 +0100 Subject: [PATCH 08/18] MAPREDUCE-7435. following chain through to validation * validation also uses manifest entries (and so works!) * testing expects this * tests of IOStats * tests of new RemoteIterators Change-Id: I4cfb308d4b08f1f775cfdbe2df6f8ff07ac6bc54 --- .../fs/statistics/IOStatisticsSetters.java | 4 +- .../fs/statistics/IOStatisticsSnapshot.java | 4 +- .../impl/ForwardingIOStatisticsStore.java | 3 +- .../util/functional/RemoteIterators.java | 8 +- .../statistics/TestIOStatisticsSetters.java | 172 +++++++++++++++ .../util/functional/TestRemoteIterators.java | 37 ++++ .../manifest/ManifestCommitterConfig.java | 20 +- .../manifest/ManifestCommitterConstants.java | 22 +- .../committer/manifest/impl/EntryFileIO.java | 55 +++-- .../manifest/impl/InternalConstants.java | 11 - .../manifest/impl/LoadedManifestData.java | 28 ++- .../manifest/stages/CleanupJobStage.java | 2 +- .../manifest/stages/CommitJobStage.java | 203 +++++++++--------- .../manifest/stages/LoadManifestsStage.java | 57 ++--- .../manifest/stages/RenameFilesStage.java | 28 ++- .../manifest/stages/StageConfig.java | 26 ++- .../stages/ValidateRenamedFilesStage.java | 48 ++--- .../src/site/markdown/manifest_committer.md | 58 ++++- .../lib/output/TestFileOutputCommitter.java | 1 - .../AbstractManifestCommitterTest.java | 4 +- .../TestJobThroughManifestCommitter.java | 49 ++--- .../manifest/TestLoadManifestsStage.java | 11 +- .../manifest/impl/TestEntryFileIO.java | 35 ++- .../hadoop/fs/s3a/S3AInstrumentation.java | 2 +- 24 files changed, 626 insertions(+), 262 deletions(-) rename {hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a => hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs}/statistics/impl/ForwardingIOStatisticsStore.java (97%) create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsSetters.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSetters.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSetters.java index 8ad137541c146..402ea27235408 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSetters.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSetters.java @@ -29,11 +29,11 @@ * {@link IOStatisticsSnapshot} to also support it. * These are the simple setters, they don't provide for increments, * decrements, calculation of min/max/mean etc. - * @since The interface and IOStatisticsSnapshot support came after Hadoop 3.3.5 + * @since The interface and IOStatisticsSnapshot support was added after Hadoop 3.3.5 */ @InterfaceAudience.Public @InterfaceStability.Evolving -public interface IOStatisticsSetters { +public interface IOStatisticsSetters extends IOStatistics { /** * Set a counter. 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 index e855a3f611e4d..4551c97665ba3 100644 --- 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 @@ -63,7 +63,7 @@ @InterfaceStability.Evolving public final class IOStatisticsSnapshot implements IOStatistics, Serializable, IOStatisticsAggregator, - IOStatisticsSetters{ + IOStatisticsSetters { private static final long serialVersionUID = -1762522703841538084L; @@ -247,7 +247,7 @@ public synchronized void setMinimum(final String key, final long value) { @Override public void setMeanStatistic(final String key, final MeanStatistic value) { - + meanStatistics().put(key, value); } @Override diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/ForwardingIOStatisticsStore.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/ForwardingIOStatisticsStore.java similarity index 97% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/ForwardingIOStatisticsStore.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/ForwardingIOStatisticsStore.java index 612460835403e..dc6546ae17323 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/ForwardingIOStatisticsStore.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/ForwardingIOStatisticsStore.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.fs.s3a.statistics.impl; +package org.apache.hadoop.fs.statistics.impl; import javax.annotation.Nullable; import java.time.Duration; @@ -25,7 +25,6 @@ import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.MeanStatistic; -import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; /** * This may seem odd having an IOStatisticsStore which does nothing 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 index f82c7e281ea61..3140e5eb74d80 100644 --- 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 @@ -739,11 +739,17 @@ public void close() throws IOException { private static final class HaltableRemoteIterator extends WrappingRemoteIterator { + /** + * Probe as to whether work should continue. + */ private final CallableRaisingIOE continueWork; - /** * Wrap an iterator with one which adds a continuation probe. + * The probe will be called in the {@link #hasNext()} method, before + * the source iterator is itself checked and in {@link #next()} + * before retrieval. + * That is: it may be called multiple times per iteration. * @param source source iterator. * @param continueWork predicate which will trigger a fast halt if it returns false. */ diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsSetters.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsSetters.java new file mode 100644 index 0000000000000..e8fc033f10ec7 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsSetters.java @@ -0,0 +1,172 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics; + +import java.util.Arrays; +import java.util.Collection; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.hadoop.fs.statistics.impl.ForwardingIOStatisticsStore; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticGauge; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticMaximum; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticMean; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticMinimum; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore; + +/** + * Test the {@link IOStatisticsSetters} interface implementations through + * a parameterized run with each implementation. + * For each of the setters, the value is set, verified, + * updated, verified again. + * An option known to be undefined in all created IOStatisticsStore instances + * is set, to verify it is harmless. + */ + +@RunWith(Parameterized.class) + +public class TestIOStatisticsSetters extends AbstractHadoopTestBase { + + public static final String COUNTER = "counter"; + + public static final String GAUGE = "gauge"; + + public static final String MAXIMUM = "max"; + + public static final String MINIMUM = "min"; + + public static final String MEAN = "mean"; + + private final IOStatisticsSetters ioStatistics; + + @Parameterized.Parameters + public static Collection params() { + return Arrays.asList(new Object[][]{ + {new IOStatisticsSnapshot()}, + {createTestStore()}, + {new ForwardingIOStatisticsStore(createTestStore())}, + }); + } + + /** + * Create a test store with the stats used for testing set up. + * @return a set up store + */ + private static IOStatisticsStore createTestStore() { + return iostatisticsStore() + .withCounters(COUNTER) + .withGauges(GAUGE) + .withMaximums(MAXIMUM) + .withMinimums(MINIMUM) + .withMeanStatistics(MEAN) + .build(); + } + + public TestIOStatisticsSetters(IOStatisticsSetters ioStatisticsSetters) { + this.ioStatistics = ioStatisticsSetters; + } + + @Test + public void testCounter() throws Throwable { + // write + ioStatistics.setCounter(COUNTER, 1); + assertThatStatisticCounter(ioStatistics, COUNTER) + .isEqualTo(1); + + // update + ioStatistics.setCounter(COUNTER, 2); + assertThatStatisticCounter(ioStatistics, COUNTER) + .isEqualTo(2); + + // unknown value + ioStatistics.setCounter("c2", 3); + } + + @Test + public void testMaximum() throws Throwable { + // write + ioStatistics.setMaximum(MAXIMUM, 1); + assertThatStatisticMaximum(ioStatistics, MAXIMUM) + .isEqualTo(1); + + // update + ioStatistics.setMaximum(MAXIMUM, 2); + assertThatStatisticMaximum(ioStatistics, MAXIMUM) + .isEqualTo(2); + + // unknown value + ioStatistics.setMaximum("mm2", 3); + } + + @Test + public void testMinimum() throws Throwable { + // write + ioStatistics.setMinimum(MINIMUM, 1); + assertThatStatisticMinimum(ioStatistics, MINIMUM) + .isEqualTo(1); + + // update + ioStatistics.setMinimum(MINIMUM, 2); + assertThatStatisticMinimum(ioStatistics, MINIMUM) + .isEqualTo(2); + + // unknown value + ioStatistics.setMinimum("c2", 3); + } + + @Test + public void testGauge() throws Throwable { + // write + ioStatistics.setGauge(GAUGE, 1); + assertThatStatisticGauge(ioStatistics, GAUGE) + .isEqualTo(1); + + // update + ioStatistics.setGauge(GAUGE, 2); + assertThatStatisticGauge(ioStatistics, GAUGE) + .isEqualTo(2); + + // unknown value + ioStatistics.setGauge("g2", 3); + } + + @Test + public void testMean() throws Throwable { + // write + final MeanStatistic mean11 = new MeanStatistic(1, 1); + ioStatistics.setMeanStatistic(MEAN, mean11); + assertThatStatisticMean(ioStatistics, MEAN) + .isEqualTo(mean11); + + // update + final MeanStatistic mean22 = new MeanStatistic(2, 2); + ioStatistics.setMeanStatistic(MEAN, mean22); + assertThatStatisticMean(ioStatistics, MEAN) + .isEqualTo(mean22); + + // unknown value + ioStatistics.setMeanStatistic("m2", mean11); + } +} 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 index 7797955ebb2f2..a95360b9a0d8d 100644 --- 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 @@ -22,8 +22,11 @@ import java.io.IOException; import java.util.Iterator; import java.util.NoSuchElementException; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.util.Preconditions; + +import org.assertj.core.api.Assertions; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,6 +40,7 @@ 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.*; +import static org.apache.hadoop.util.functional.RemoteIterators.haltableRemoteIterator; import static org.assertj.core.api.Assertions.assertThat; /** @@ -287,6 +291,39 @@ public void testJavaIterableCloseInNextLoop() throws Throwable { } + @Test + public void testHaltableIterator() throws Throwable { + final int limit = 4; + AtomicInteger count = new AtomicInteger(limit); + + // a countdown of 10, but the halting predicate will fail earlier + // if the value of "count" has dropped to zero + final RemoteIterator it = + haltableRemoteIterator( + new CountdownRemoteIterator(10), + () -> count.get() > 0); + + Assertions.assertThat(foreach(it, + (v) -> count.decrementAndGet())) + .describedAs("Count of iterations") + .isEqualTo(limit); + } + @Test + public void testHaltableIteratorNoHalt() throws Throwable { + + // a countdown of 10, but the halting predicate will fail earlier + // if the value of "count" has dropped to zero + final RemoteIterator it = + haltableRemoteIterator( + new CountdownRemoteIterator(10), + () -> true); + + Assertions.assertThat(foreach(it, + (v) -> {})) + .describedAs("Count of iterations") + .isEqualTo(10); + } + /** * assert that the string value of an object contains the * expected text. diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConfig.java index eb9c502129f72..8a1ae0fcc9810 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConfig.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConfig.java @@ -148,6 +148,11 @@ public final class ManifestCommitterConfig implements IOStatisticsSource { */ private final boolean deleteTargetPaths; + /** + * Entry writer queue capacity. + */ + private final int writerQueueCapacity; + /** * Constructor. * @param outputPath destination path of the job. @@ -190,6 +195,9 @@ public final class ManifestCommitterConfig implements IOStatisticsSource { this.deleteTargetPaths = conf.getBoolean( OPT_DELETE_TARGET_FILES, OPT_DELETE_TARGET_FILES_DEFAULT); + this.writerQueueCapacity = conf.getInt( + OPT_WRITER_QUEUE_CAPACITY, + DEFAULT_WRITER_QUEUE_CAPACITY); // if constructed with a task attempt, build the task ID and path. if (context instanceof TaskAttemptContext) { @@ -252,6 +260,7 @@ StageConfig createStageConfig() { StageConfig stageConfig = new StageConfig(); stageConfig .withConfiguration(conf) + .withDeleteTargetPaths(deleteTargetPaths) .withIOStatistics(iostatistics) .withJobAttemptNumber(jobAttemptNumber) .withJobDirectories(dirs) @@ -263,8 +272,7 @@ StageConfig createStageConfig() { .withTaskAttemptDir(taskAttemptDir) .withTaskAttemptId(taskAttemptId) .withTaskId(taskId) - .withDeleteTargetPaths(deleteTargetPaths); - + .withWriterQueueCapacity(writerQueueCapacity); return stageConfig; } @@ -324,6 +332,14 @@ public String getName() { return name; } + /** + * Get writer queue capacity. + * @return the queue capacity + */ + public int getWriterQueueCapacity() { + return writerQueueCapacity; + } + @Override public IOStatisticsStore getIOStatistics() { return iostatistics; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConstants.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConstants.java index fd7b3d816c103..d2a39a50cadb5 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConstants.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConstants.java @@ -151,7 +151,7 @@ public final class ManifestCommitterConstants { /** * Default value: {@value}. */ - public static final int OPT_IO_PROCESSORS_DEFAULT = 64; + public static final int OPT_IO_PROCESSORS_DEFAULT = 32; /** * Directory for saving job summary reports. @@ -240,6 +240,26 @@ public final class ManifestCommitterConstants { public static final String CAPABILITY_DYNAMIC_PARTITIONING = "mapreduce.job.committer.dynamic.partitioning"; + + /** + * Queue capacity between task manifest loading an entry file writer. + * If more than this number of manifest lists are waiting to be written, + * the enqueue is blocking. + * There's an expectation that writing to the local file is a lot faster + * than the parallelized buffer reads, therefore that this queue can + * be emptied at the same rate it is filled. + * Value {@value}. + */ + public static final String OPT_WRITER_QUEUE_CAPACITY = + OPT_PREFIX + "writer.queue.capacity"; + + + /** + * Default value of {@link #OPT_WRITER_QUEUE_CAPACITY}. + * Value {@value}. + */ + public static final int DEFAULT_WRITER_QUEUE_CAPACITY = 32; + private ManifestCommitterConstants() { } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java index aee6a314dcbe1..f3d3c114f405b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java @@ -34,6 +34,7 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,11 +46,11 @@ import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry; -import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.util.concurrent.HadoopExecutors; import org.apache.hadoop.util.functional.FutureIO; import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.util.Preconditions.checkState; /** * Read or write entry file. @@ -67,7 +68,7 @@ public class EntryFileIO { private final Configuration conf; /** - * ctor. + * Constructor. * @param conf Configuration used to load filesystems */ public EntryFileIO(final Configuration conf) { @@ -78,12 +79,18 @@ public EntryFileIO(final Configuration conf) { * Create a writer to a local file. * @param file file * @return the writer - * @throws IOException fail to open the file + * @throws IOException failure to create the file */ public SequenceFile.Writer createWriter(File file) throws IOException { return createWriter(toPath(file)); } + /** + * Create a writer to a file on any FS. + * @param path path to write to. + * @return the writer + * @throws IOException failure to create the file + */ public SequenceFile.Writer createWriter(Path path) throws IOException { return SequenceFile.createWriter(conf, SequenceFile.Writer.file(path), @@ -170,10 +177,13 @@ public static Path toPath(final File file) { return new Path(file.toURI()); } - /** - * Writer takes a list of entries at a time; queues for writing. - * A special + * A Writer thread takes reads from a queue containing + * list of entries to save; these are serialized via the writer to + * the output stream. + * Other threads can queue the file entry lists from loaded manifests + * for them to be written. + * The these threads will be blocked when the queue capacity is reached. */ public final class EntryWriter implements Closeable { @@ -205,9 +215,11 @@ public final class EntryWriter implements Closeable { private final AtomicInteger count = new AtomicInteger(); /** - * any failure. + * Any failure caught on the writer thread; this should be + * raised within the task/job thread as it implies that the + * entire write has failed. */ - private volatile IOException failure; + private final AtomicReference failure = new AtomicReference<>(); /** * Create. @@ -215,7 +227,8 @@ public final class EntryWriter implements Closeable { * @param capacity capacity. */ private EntryWriter(SequenceFile.Writer writer, int capacity) { - this.writer = writer; + checkState(capacity > 0, "invalid queue capacity %s", capacity); + this.writer = requireNonNull(writer); this.queue = new ArrayBlockingQueue<>(capacity); } @@ -240,14 +253,14 @@ public int getCount() { * @return any IOException caught when writing the output */ public IOException getFailure() { - return failure; + return failure.get(); } /** * Start the thread. */ private void start() { - Preconditions.checkState(executor == null, "already started"); + checkState(executor == null, "already started"); active.set(true); executor = HadoopExecutors.newSingleThreadExecutor(); future = executor.submit(this::processor); @@ -322,7 +335,7 @@ private void append(FileEntry entry) { LOG.trace("Added entry #{}: {}", c, entry); } catch (IOException e) { LOG.debug("Write failure", e); - failure = e; + failure.set(e); throw new UncheckedIOException(e); } } @@ -369,8 +382,9 @@ public void close() throws IOException { * @throws IOException if one was caught and saved. */ public void maybeRaiseWriteException() throws IOException { - if (failure != null) { - throw failure; + final IOException f = failure.get(); + if (f != null) { + throw f; } } @@ -392,7 +406,7 @@ public String toString() { * No thread safety. */ @VisibleForTesting - final class EntryIterator implements RemoteIterator, Closeable { + static final class EntryIterator implements RemoteIterator, Closeable { private final SequenceFile.Reader reader; @@ -402,6 +416,10 @@ final class EntryIterator implements RemoteIterator, Closeable { private int count; + /** + * Create an iterator. + * @param reader the file to read from. + */ private EntryIterator(final SequenceFile.Reader reader) { this.reader = requireNonNull(reader); } @@ -428,6 +446,13 @@ public boolean hasNext() throws IOException { return fetched != null || fetchNext(); } + /** + * Fetch the next entry. + * If there is none, then the reader is closed before `false` + * is returned. + * @return true if a record was retrieved. + * @throws IOException IO failure. + */ private boolean fetchNext() throws IOException { FileEntry readBack = new FileEntry(); if (reader.next(NullWritable.get(), readBack)) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/InternalConstants.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/InternalConstants.java index 804cbde01f36f..9f4fcde10857c 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/InternalConstants.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/InternalConstants.java @@ -129,15 +129,4 @@ private InternalConstants() { public static final Set UNSUPPORTED_FS_SCHEMAS = ImmutableSet.of("s3a", "wasb"); - /** - * Queue capacity between task manifest loading an entry file writer. - * If more than this number of manifest lists are waiting to be written, - * the enqueue is blocking. - * There's an expectation that writing to the local file is a lot faster - * than the parallelized buffer reads, therefore that this queue can - * be emptied at the same rate it is filled. - * Value {@value}. - */ - public static final int ENTRY_WRITER_QUEUE_CAPACITY = 32; - } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/LoadedManifestData.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/LoadedManifestData.java index 690eff761811c..d816c817599d9 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/LoadedManifestData.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/LoadedManifestData.java @@ -25,6 +25,8 @@ import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DirEntry; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.LoadManifestsStage; +import static java.util.Objects.requireNonNull; + /** * Information about the loaded manifest data; * Returned from {@link LoadManifestsStage} and then @@ -49,13 +51,19 @@ public final class LoadedManifestData { */ private final int fileCount; + /** + * Data about the loaded manifests + * @param directories directories + * @param entrySequenceData Path in local fs to the entry sequence data. + * @param fileCount number of files. + */ public LoadedManifestData( final Collection directories, - final Path entrySequenceFile, + final Path entrySequenceData, final int fileCount) { - this.directories = directories; + this.directories = requireNonNull(directories); this.fileCount = fileCount; - this.entrySequenceData = entrySequenceFile; + this.entrySequenceData = requireNonNull(entrySequenceData); } public Collection getDirectories() { @@ -66,11 +74,25 @@ public int getFileCount() { return fileCount; } + /** + * Get the path to the entry sequence data file. + * @return the path + */ public Path getEntrySequenceData() { return entrySequenceData; } + /** + * Get the entry sequence data as a file; + */ public File getEntrySequenceFile() { return new File(entrySequenceData.toUri()); } + + /** + * Delete the entry sequence file. + */ + public void deleteEntrySequenceFile() { + getEntrySequenceFile().delete(); + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CleanupJobStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CleanupJobStage.java index 4a5971a5b1cb0..77b80aaf67fd6 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CleanupJobStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CleanupJobStage.java @@ -295,7 +295,7 @@ public static final class Arguments { * @param statisticName stage name to report * @param enabled is the stage enabled? * @param deleteTaskAttemptDirsInParallel delete task attempt dirs in - * parallel? + * parallel? * @param suppressExceptions suppress exceptions? */ public Arguments( diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java index 7ecd16e46db77..02a9968407d29 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java @@ -20,7 +20,6 @@ import java.io.File; import java.io.IOException; -import java.util.List; import javax.annotation.Nullable; @@ -32,7 +31,6 @@ import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData; -import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.LoadedManifestData; import static java.util.Objects.requireNonNull; @@ -45,7 +43,6 @@ 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_RENAME_FILES; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DiagnosticKeys.MANIFESTS; -import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.InternalConstants.ENTRY_WRITER_QUEUE_CAPACITY; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.addHeapInformation; /** @@ -75,108 +72,118 @@ protected CommitJobStage.Result executeStage( getJobId(), storeSupportsResilientCommit()); - boolean createMarker = arguments.isCreateMarker(); - IOStatisticsSnapshot heapInfo = new IOStatisticsSnapshot(); - addHeapInformation(heapInfo, "setup"); - // load the manifests - final StageConfig stageConfig = getStageConfig(); - LoadManifestsStage.Result result = new LoadManifestsStage(stageConfig).apply( - new LoadManifestsStage.Arguments( - File.createTempFile("manifest", ".list"), - false, - ENTRY_WRITER_QUEUE_CAPACITY)); - LoadManifestsStage.SummaryInfo summary = result.getSummary(); - final LoadedManifestData manifestData = result.getLoadedManifestData(); - - LOG.debug("{}: Job Summary {}", getName(), summary); - LOG.info("{}: Committing job with file count: {}; total size {} bytes", - getName(), - summary.getFileCount(), - byteCountToDisplaySize(summary.getTotalFileSize())); - addHeapInformation(heapInfo, OP_STAGE_JOB_LOAD_MANIFESTS); - - - // add in the manifest statistics to our local IOStatistics for - // reporting. - IOStatisticsStore iostats = getIOStatistics(); - iostats.aggregate(summary.getIOStatistics()); - - // prepare destination directories. - final CreateOutputDirectoriesStage.Result dirStageResults = - new CreateOutputDirectoriesStage(stageConfig) - .apply(manifestData.getDirectories()); - addHeapInformation(heapInfo, OP_STAGE_JOB_CREATE_TARGET_DIRS); - - // commit all the tasks. - // The success data includes a snapshot of the IO Statistics - // and hence all aggregate stats from the tasks. - ManifestSuccessData successData; - successData = new RenameFilesStage(stageConfig).apply( - Pair.of(manifestData, dirStageResults.getCreatedDirectories())); - if (LOG.isDebugEnabled()) { - LOG.debug("{}: _SUCCESS file summary {}", getName(), successData.toJson()); - } - addHeapInformation(heapInfo, OP_STAGE_JOB_RENAME_FILES); - - // update the counter of bytes committed and files. - // use setCounter so as to ignore any values accumulated when - // aggregating tasks. - iostats.setCounter( - COMMITTER_FILES_COMMITTED_COUNT, - summary.getFileCount()); - iostats.setCounter( - COMMITTER_BYTES_COMMITTED_COUNT, - summary.getTotalFileSize()); - successData.snapshotIOStatistics(iostats); - successData.getIOStatistics().aggregate(heapInfo); - - - - // rename manifests. Only warn on failure here. - final String manifestRenameDir = arguments.getManifestRenameDir(); - if (isNotBlank(manifestRenameDir)) { - Path manifestRenamePath = new Path( - new Path(manifestRenameDir), - getJobId()); - LOG.info("{}: Renaming manifests to {}", getName(), manifestRenamePath); - try { - renameDir(getTaskManifestDir(), manifestRenamePath); - - // save this path in the summary diagnostics - successData.getDiagnostics().put(MANIFESTS, manifestRenamePath.toUri().toString()); - } catch (IOException | IllegalArgumentException e) { - // rename failure, including path for wrong filesystem - LOG.warn("{}: Failed to rename manifests to {}", getName(), manifestRenamePath, e); + // once the manifest has been loaded, a temp file needs to be + // deleted; so track teh value. + LoadedManifestData loadedManifestData = null; + + try { + boolean createMarker = arguments.isCreateMarker(); + IOStatisticsSnapshot heapInfo = new IOStatisticsSnapshot(); + addHeapInformation(heapInfo, "setup"); + // load the manifests + final StageConfig stageConfig = getStageConfig(); + LoadManifestsStage.Result result = new LoadManifestsStage(stageConfig).apply( + new LoadManifestsStage.Arguments( + File.createTempFile("manifest", ".list"), + false, /* do not cache manifests */ + stageConfig.getWriterQueueCapacity())); + LoadManifestsStage.SummaryInfo summary = result.getSummary(); + loadedManifestData = result.getLoadedManifestData(); + + LOG.debug("{}: Job Summary {}", getName(), summary); + LOG.info("{}: Committing job with file count: {}; total size {} bytes", + getName(), + summary.getFileCount(), + byteCountToDisplaySize(summary.getTotalFileSize())); + addHeapInformation(heapInfo, OP_STAGE_JOB_LOAD_MANIFESTS); + + + // add in the manifest statistics to our local IOStatistics for + // reporting. + IOStatisticsStore iostats = getIOStatistics(); + iostats.aggregate(summary.getIOStatistics()); + + // prepare destination directories. + final CreateOutputDirectoriesStage.Result dirStageResults = + new CreateOutputDirectoriesStage(stageConfig) + .apply(loadedManifestData.getDirectories()); + addHeapInformation(heapInfo, OP_STAGE_JOB_CREATE_TARGET_DIRS); + + // commit all the tasks. + // The success data includes a snapshot of the IO Statistics + // and hence all aggregate stats from the tasks. + ManifestSuccessData successData; + successData = new RenameFilesStage(stageConfig).apply( + Pair.of(loadedManifestData, dirStageResults.getCreatedDirectories())); + if (LOG.isDebugEnabled()) { + LOG.debug("{}: _SUCCESS file summary {}", getName(), successData.toJson()); + } + addHeapInformation(heapInfo, OP_STAGE_JOB_RENAME_FILES); + + // update the counter of bytes committed and files. + // use setCounter so as to ignore any values accumulated when + // aggregating tasks. + iostats.setCounter( + COMMITTER_FILES_COMMITTED_COUNT, + summary.getFileCount()); + iostats.setCounter( + COMMITTER_BYTES_COMMITTED_COUNT, + summary.getTotalFileSize()); + successData.snapshotIOStatistics(iostats); + successData.getIOStatistics().aggregate(heapInfo); + + // rename manifests. Only warn on failure here. + final String manifestRenameDir = arguments.getManifestRenameDir(); + if (isNotBlank(manifestRenameDir)) { + Path manifestRenamePath = new Path( + new Path(manifestRenameDir), + getJobId()); + LOG.info("{}: Renaming manifests to {}", getName(), manifestRenamePath); + try { + renameDir(getTaskManifestDir(), manifestRenamePath); + + // save this path in the summary diagnostics + successData.getDiagnostics().put(MANIFESTS, manifestRenamePath.toUri().toString()); + } catch (IOException | IllegalArgumentException e) { + // rename failure, including path for wrong filesystem + LOG.warn("{}: Failed to rename manifests to {}", getName(), manifestRenamePath, e); + } } - } - // save the _SUCCESS if the option is enabled. - Path successPath = null; - if (createMarker) { - // save a snapshot of the IO Statistics + // save the _SUCCESS if the option is enabled. + Path successPath = null; + if (createMarker) { + // save a snapshot of the IO Statistics - successPath = new SaveSuccessFileStage(stageConfig) - .apply(successData); - LOG.debug("{}: Saving _SUCCESS file to {}", getName(), successPath); - } + successPath = new SaveSuccessFileStage(stageConfig) + .apply(successData); + LOG.debug("{}: Saving _SUCCESS file to {}", getName(), successPath); + } - // optional cleanup - new CleanupJobStage(stageConfig).apply(arguments.getCleanupArguments()); + // optional cleanup + new CleanupJobStage(stageConfig).apply(arguments.getCleanupArguments()); - // and then, after everything else: optionally validate. - if (arguments.isValidateOutput()) { - // cache and restore the active stage field - LOG.info("{}: Validating output.", getName()); - new ValidateRenamedFilesStage(stageConfig) - .apply(result.getManifests()); - } + // and then, after everything else: optionally validate. + if (arguments.isValidateOutput()) { + // cache and restore the active stage field + LOG.info("{}: Validating output.", getName()); + new ValidateRenamedFilesStage(stageConfig) + .apply(loadedManifestData.getEntrySequenceData()); + } - // restore the active stage so that when the report is saved - // it is declared as job commit, not cleanup or validate. - stageConfig.enterStage(getStageName(arguments)); + // restore the active stage so that when the report is saved + // it is declared as job commit, not cleanup or validate. + stageConfig.enterStage(getStageName(arguments)); - // the result - return new CommitJobStage.Result(successPath, successData); + // the result + return new Result(successPath, successData); + } finally { + // cleanup + if (loadedManifestData != null) { + loadedManifestData.deleteEntrySequenceFile(); + } + + } } /** diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java index 95b33c602511c..7c8e84a5a2319 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java @@ -75,9 +75,6 @@ public class LoadManifestsStage extends */ private final SummaryInfo summaryInfo = new SummaryInfo(); - - - /** * List of loaded manifests. */ @@ -121,28 +118,35 @@ protected LoadManifestsStage.Result executeStage( manifestDir); cacheManifests = arguments.cacheManifests; - final Path entrySequenceFile = arguments.getEntrySequenceFile(); + final Path entrySequenceData = arguments.getEntrySequenceData(); // the entry writer for queuing data. entryWriter = entryFileIO.launchEntryWriter( - entryFileIO.createWriter(entrySequenceFile), + entryFileIO.createWriter(entrySequenceData), arguments.queueCapacity); + // manifest list is only built up when caching is enabled. + // as this is memory hungry, it is warned about List manifestList; + if (arguments.cacheManifests) { + LOG.info("Loaded manifests are cached; this is memory hungry"); + } try { - // sync fs before the listj + // 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 manifestFiles = - haltableRemoteIterator(listManifests(), () -> entryWriter.isActive()); + haltableRemoteIterator(listManifests(), + () -> entryWriter.isActive()); manifestList = loadAllManifests(manifestFiles); + maybeAddIOStatistics(getIOStatistics(), manifestFiles); LOG.info("{}: Summary of {} manifests loaded in {}: {}", getName(), - manifestList.size(), + summaryInfo.manifestCount, manifestDir, summaryInfo); @@ -153,13 +157,12 @@ protected LoadManifestsStage.Result executeStage( entryWriter.maybeRaiseWriteException(); // collect any stats - maybeAddIOStatistics(getIOStatistics(), manifestFiles); } finally { entryWriter.close(); } final LoadedManifestData loadedManifestData = new LoadedManifestData( new ArrayList<>(directories.values()), // new array to free up the map - entrySequenceFile, + entrySequenceData, entryWriter.getCount()); return new LoadManifestsStage.Result(summaryInfo, loadedManifestData, manifestList); @@ -196,30 +199,35 @@ private void processOneManifest(FileStatus status) // update the directories final int created = coalesceDirectories(manifest); + final String taskID = manifest.getTaskID(); LOG.debug("{}: task {} added {} directories", - getName(), manifest.getTaskID(), created); - - // queue those files. - final boolean enqueued = entryWriter.enqueue(manifest.getFilesToCommit()); - if (!enqueued) { - LOG.warn("{}: Failed to write manifest for task {}", - getName(), - manifest.getTaskID()); - } + getName(), taskID, created); // add to the summary. summaryInfo.add(manifest); - // if manifests are cached, clear extra data - // and then save. + // clear the manifest extra data so if + // blocked waiting for queue capacity, + // memory use is reduced. + manifest.setIOStatistics(null); + manifest.getExtraData().clear(); + + // if manifests are cached add to the list if (cacheManifests) { - manifest.setIOStatistics(null); - manifest.getExtraData().clear(); // update the manifest list in a synchronized block. synchronized (manifests) { manifests.add(manifest); } } + + // queue those files. + final boolean enqueued = entryWriter.enqueue(manifest.getFilesToCommit()); + if (!enqueued) { + LOG.warn("{}: Failed to write manifest for task {}", + getName(), + taskID); + } + } /** @@ -250,7 +258,6 @@ int coalesceDirectories(final TaskManifest manifest) { } } return toCreate.size(); - } /** @@ -311,7 +318,7 @@ public Arguments(final File entrySequenceFile, this.queueCapacity = queueCapacity; } - private Path getEntrySequenceFile() { + private Path getEntrySequenceData() { return new Path(entrySequenceFile.toURI()); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/RenameFilesStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/RenameFilesStage.java index 2bdf7f165a331..a8cd74e938776 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/RenameFilesStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/RenameFilesStage.java @@ -18,7 +18,6 @@ package org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages; -import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -30,7 +29,6 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData; @@ -45,8 +43,15 @@ /** * This stage renames all the files. - * Input: the manifests and the set of directories created, as returned by - * {@link CreateOutputDirectoriesStage}. + * Input: + *

    + *
  1. {@link LoadedManifestData} from the {@link LoadManifestsStage}
  2. + *
  3. the set of directories created, as returned by + * {@link CreateOutputDirectoriesStage}.
  4. + *
+ * The files to rename are determined by reading the entry file referenced + * in the {@link LoadedManifestData}; these are read and renamed incrementally. + * * If the job is configured to delete target files, if the parent dir * had to be created, the delete() call can be skipped. * It returns a manifest success data file summarizing the @@ -108,8 +113,6 @@ protected ManifestSuccessData executeStage( final LoadedManifestData manifestData = args.getLeft(); createdDirectories = args.getRight(); final EntryFileIO entryFileIO = new EntryFileIO(getStageConfig().getConf()); - final SequenceFile.Reader reader = - entryFileIO.createReader(manifestData.getEntrySequenceData()); final ManifestSuccessData success = createManifestOutcome(getStageConfig(), @@ -118,19 +121,14 @@ protected ManifestSuccessData executeStage( LOG.info("{}: Executing Manifest Job Commit with {} files", getName(), manifestData.getFileCount()); - // iterate over the + // iterate over the entries in the file. + try (SequenceFile.Reader reader = entryFileIO.createReader( + manifestData.getEntrySequenceData())) { - final RemoteIterator entries = entryFileIO.iterateOver(reader); - try { - TaskPool.foreach(entries) + TaskPool.foreach(entryFileIO.iterateOver(reader)) .executeWith(getIOProcessors()) .stopOnFailure() .run(this::commitOneFile); - } finally { - // close the input. Automatic on a successful scan, but - // a rename failure will abort the operation. - ((Closeable)entries).close(); - } // synchronized block to keep spotbugs happy. diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/StageConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/StageConfig.java index 59d9c7faae626..9ad121c2761c2 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/StageConfig.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/StageConfig.java @@ -29,6 +29,7 @@ import org.apache.hadoop.util.Progressable; import org.apache.hadoop.util.functional.TaskPool; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.DEFAULT_WRITER_QUEUE_CAPACITY; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SUCCESS_MARKER; /** @@ -156,10 +157,15 @@ public class StageConfig { /** * Configuration used where needed. * Default value is a configuration with the normal constructor; - * jobs should override this with what was passed down.j + * jobs should override this with what was passed down. */ private Configuration conf = new Configuration(); + /** + * Entry writer queue capacity. + */ + private int writerQueueCapacity = DEFAULT_WRITER_QUEUE_CAPACITY; + public StageConfig() { } @@ -431,6 +437,24 @@ public Configuration getConf() { return conf; } + /** + * Get writer queue capacity. + * @return the queue capacity + */ + public int getWriterQueueCapacity() { + return writerQueueCapacity; + } + + /** + * Set writer queue capacity. + * @param value new value + * @return the builder + */ + public StageConfig withWriterQueueCapacity(final int value) { + writerQueueCapacity = value; + return this; + } + /** * Handler for stage entry events. * @return the handler. diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/ValidateRenamedFilesStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/ValidateRenamedFilesStage.java index 8b72112a3dad8..37c44a71f2972 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/ValidateRenamedFilesStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/ValidateRenamedFilesStage.java @@ -22,23 +22,21 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.stream.Collectors; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry; -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.ManifestCommitterSupport; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.OutputValidationException; import org.apache.hadoop.util.functional.TaskPool; import static org.apache.commons.lang3.StringUtils.isNotBlank; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_VALIDATE_OUTPUT; -import static org.apache.hadoop.thirdparty.com.google.common.collect.Iterables.concat; /** * This stage validates all files by scanning the manifests @@ -50,17 +48,12 @@ */ public class ValidateRenamedFilesStage extends AbstractJobOrTaskStage< - List, + Path, List> { private static final Logger LOG = LoggerFactory.getLogger( ValidateRenamedFilesStage.class); - /** - * Set this to halt all workers. - */ - private final AtomicBoolean halt = new AtomicBoolean(); - /** * List of all files committed. */ @@ -93,34 +86,27 @@ private synchronized void addFileCommitted(FileEntry entry) { * has a file in the destination of the same size. * If two tasks have both written the same file or * a source file was changed after the task was committed, - * then a mistmatch will be detected -provided the file + * then a mismatch will be detected -provided the file * length is now different. - * @param taskManifests list of manifests. + * @param entryFile path to entry file * @return list of files committed. */ @Override protected List executeStage( - final List taskManifests) + final Path entryFile) throws IOException { - // set the list of files to be as big as the number of tasks. - // synchronized to stop complaints. - synchronized (this) { - filesCommitted = new ArrayList<>(taskManifests.size()); - } - - // validate all the files. - - final Iterable filesToCommit = concat(taskManifests.stream() - .map(TaskManifest::getFilesToCommit) - .collect(Collectors.toList())); + final EntryFileIO entryFileIO = new EntryFileIO(getStageConfig().getConf()); - TaskPool.foreach(filesToCommit) - .executeWith(getIOProcessors()) - .stopOnFailure() - .run(this::validateOneFile); + try (SequenceFile.Reader reader = entryFileIO.createReader(entryFile)) { + // iterate over the entries in the file. + TaskPool.foreach(entryFileIO.iterateOver(reader)) + .executeWith(getIOProcessors()) + .stopOnFailure() + .run(this::validateOneFile); - return getFilesCommitted(); + return getFilesCommitted(); + } } /** @@ -132,10 +118,6 @@ protected List executeStage( private void validateOneFile(FileEntry entry) throws IOException { updateAuditContext(OP_STAGE_JOB_VALIDATE_OUTPUT); - if (halt.get()) { - // told to stop - return; - } // report progress back progress(); // look validate the file. diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer.md b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer.md index c95486549db5d..bf37feff408ec 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer.md +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer.md @@ -207,17 +207,18 @@ in the option `mapreduce.manifest.committer.io.threads`. Larger values may be used. -XML +Hadoop XML configuration ```xml mapreduce.manifest.committer.io.threads - 200 + 32 ``` -spark-defaults.conf -``` -spark.hadoop.mapreduce.manifest.committer.io.threads 200 +In `spark-defaults.conf` + +```properties +spark.hadoop.mapreduce.manifest.committer.io.threads 32 ``` A larger value than that of the number of cores allocated to @@ -225,6 +226,10 @@ the MapReduce AM or Spark Driver does not directly overload the CPUs, as the threads are normally waiting for (slow) IO against the object store/filesystem to complete. +Manifest loading in job commit may be memory intensive; +the larger the number of threads, the more manifests which +will be loaded simultaneously. + Caveats * In Spark, multiple jobs may be committed in the same process, each of which will create their own thread pool during job @@ -234,6 +239,36 @@ Caveats `mapreduce.manifest.committer.io.rate` can help avoid this. +### `mapreduce.manifest.committer.writer.queue.capacity` + +This is a secondary scale option. +It controls the size of the queue for storing lists of files to rename from +the manifests loaded from the target filesystem, manifests loaded +from a pool of worker threads, and the single thread which saves +the entries from each manifest to an intermediate file in the local filesystem. + +Once the queue is full, all manifest loading threads will block. + +```xml + + mapreduce.manifest.committer.writer.queue.capacity + 32 + +``` + +As the local filesystem is usually much faster to write to than any cloud store, +this queue size should not be a limit on manifest load performance. + +It can help limit the amount of memory consumed during manifest load during +job commit. +The maximumum number of loaded manifests will be + +``` +mapreduce.manifest.committer.writer.queue.capacity + mapreduce.manifest.committer.io.threads +``` + + + ## Optional: deleting target files in Job Commit The classic `FileOutputCommitter` deletes files at the destination paths @@ -611,13 +646,14 @@ spark.hadoop.mapreduce.manifest.committer.summary.report.directory (optional: U There are some advanced options which are intended for development and testing, rather than production use. -| Option | Meaning | Default Value | -|--------|---------|---------------| -| `mapreduce.manifest.committer.store.operations.classname` | Classname for Manifest Store Operations | `""` | -| `mapreduce.manifest.committer.validate.output` | Perform output validation? | `false` | +| Option | Meaning | Default Value | +|--------|----------------------------------------------|---------------| +| `mapreduce.manifest.committer.store.operations.classname` | Classname for Manifest Store Operations | `""` | +| `mapreduce.manifest.committer.validate.output` | Perform output validation? | `false` | +| `mapreduce.manifest.committer.writer.queue.capacity` | Queue capacity for writing intermediate file | `32` | -## Validating output `mapreduce.manifest.committer.validate.output` +### Validating output `mapreduce.manifest.committer.validate.output` The option `mapreduce.manifest.committer.validate.output` triggers a check of every renamed file to verify it has the expected length. @@ -626,7 +662,7 @@ This adds the overhead of a `HEAD` request per file, and so is recommended for t There is no verification of the actual contents. -## Controlling storage integration `mapreduce.manifest.committer.store.operations.classname` +### Controlling storage integration `mapreduce.manifest.committer.store.operations.classname` The manifest committer interacts with filesystems through implementations of the interface `ManifestStoreOperations`. diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java index 0cb88fa83609e..9c58c0d773f99 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java @@ -823,7 +823,6 @@ public Void call() throws IOException, InterruptedException { // needed to avoid this test contaminating others in the same JVM FileSystem.closeAll(); conf.set(fileImpl, fileImplClassname); - conf.set(fileImpl, fileImplClassname); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/AbstractManifestCommitterTest.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/AbstractManifestCommitterTest.java index 1a1e44bae4777..514f2fe8ec7ce 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/AbstractManifestCommitterTest.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/AbstractManifestCommitterTest.java @@ -74,6 +74,7 @@ import static org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.SUCCESSFUL_JOB_OUTPUT_DIR_MARKER; import static org.apache.hadoop.mapreduce.lib.output.PathOutputCommitterFactory.COMMITTER_FACTORY_CLASS; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConfig.createCloseableTaskSubmitter; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.DEFAULT_WRITER_QUEUE_CAPACITY; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.JOB_ID_SOURCE_MAPREDUCE; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.MANIFEST_COMMITTER_FACTORY; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_DIAGNOSTICS_MANIFEST_DIR; @@ -796,7 +797,8 @@ protected StageConfig createStageConfig( .withJobDirectories(attemptDirs) .withName(String.format(NAME_FORMAT_JOB_ATTEMPT, jobId)) .withOperations(getStoreOperations()) - .withProgressable(getProgressCounter()); + .withProgressable(getProgressCounter()) + .withWriterQueueCapacity(DEFAULT_WRITER_QUEUE_CAPACITY); // if there's a task attempt ID set, set up its details if (taskIndex >= 0) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java index 7de38557cdf0c..feeb20d348108 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java @@ -26,6 +26,7 @@ import java.util.stream.Collectors; import org.assertj.core.api.Assertions; +import org.assertj.core.api.Assumptions; import org.junit.FixMethodOrder; import org.junit.Test; import org.junit.runners.MethodSorters; @@ -37,7 +38,7 @@ import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; -import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.InternalConstants; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.LoadedManifestData; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.OutputValidationException; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.AbortTaskStage; @@ -55,6 +56,7 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyPathExists; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.DEFAULT_WRITER_QUEUE_CAPACITY; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.JOB_ID_SOURCE_MAPREDUCE; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.MANIFEST_COMMITTER_CLASSNAME; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_BYTES_COMMITTED_COUNT; @@ -65,7 +67,6 @@ import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterTestSupport.validateGeneratedFiles; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DiagnosticKeys.PRINCIPAL; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DiagnosticKeys.STAGE; -import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.InternalConstants.ENTRY_WRITER_QUEUE_CAPACITY; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.manifestPathForTask; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.CleanupJobStage.DISABLED; import static org.apache.hadoop.security.UserGroupInformation.getCurrentUser; @@ -144,6 +145,9 @@ public class TestJobThroughManifestCommitter */ private StageConfig ta11Config; + private LoadedManifestData + loadedManifestData; + @Override public void setup() throws Exception { super.setup(); @@ -447,9 +451,12 @@ public void test_0400_loadManifests() throws Throwable { describe("Load all manifests; committed must be TA01 and TA10"); File entryFile = File.createTempFile("entry", ".seq"); LoadManifestsStage.Arguments args = new LoadManifestsStage.Arguments( - entryFile, true, InternalConstants.ENTRY_WRITER_QUEUE_CAPACITY); + entryFile, true, DEFAULT_WRITER_QUEUE_CAPACITY); LoadManifestsStage.Result result = new LoadManifestsStage(getJobStageConfig()).apply(args); + + loadedManifestData = result.getLoadedManifestData(); + String summary = result.getSummary().toString(); LOG.info("Manifest summary {}", summary); List manifests = result.getManifests(); @@ -479,6 +486,9 @@ public void test_0410_commitJob() throws Throwable { public void test_0420_validateJob() throws Throwable { describe("Validate the output of the job through the validation" + " stage"); + Assumptions.assumeThat(loadedManifestData) + .describedAs("Loaded Manifest Data from earlier stage") + .isNotNull(); // load in the success data. @@ -486,16 +496,12 @@ public void test_0420_validateJob() throws Throwable { getFileSystem(), getJobStageConfig().getJobSuccessMarkerPath()); - // load manifests stage will load all the task manifests again - List manifests = new LoadManifestsStage(getJobStageConfig()) - .apply(new LoadManifestsStage.Arguments( - File.createTempFile("manifest", ".list"), - true, - ENTRY_WRITER_QUEUE_CAPACITY)) - .getManifests(); + // Now verify their files exist, returning the list of renamed files. - List committedFiles = new ValidateRenamedFilesStage(getJobStageConfig()) - .apply(manifests) + final List validatedEntries = new ValidateRenamedFilesStage(getJobStageConfig()) + .apply(loadedManifestData.getEntrySequenceData()); + + List committedFiles = validatedEntries .stream().map(FileEntry::getDest) .collect(Collectors.toList()); @@ -507,24 +513,11 @@ public void test_0420_validateJob() throws Throwable { Assertions.assertThat(committedFiles) .containsAll(successData.getFilenames()); - // now patch one of the manifest files by editing an entry - FileEntry entry = manifests.get(0).getFilesToCommit().get(0); - // no longer exists. - String oldName = entry.getDest(); - String newName = oldName + ".missing"; - entry.setDest(newName); - - // validation will now fail - intercept(OutputValidationException.class, ".missing", () -> - new ValidateRenamedFilesStage(getJobStageConfig()) - .apply(manifests)); - - // restore the name, but change the size - entry.setDest(oldName); - entry.setSize(128_000_000); + // delete an entry, repeat + getFileSystem().delete(validatedEntries.get(0).getDestPath()); intercept(OutputValidationException.class, () -> new ValidateRenamedFilesStage(getJobStageConfig()) - .apply(manifests)); + .apply(loadedManifestData.getEntrySequenceData())); } @Test diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestLoadManifestsStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestLoadManifestsStage.java index 9c8ab98211dfe..8a7626116fa52 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestLoadManifestsStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestLoadManifestsStage.java @@ -31,7 +31,6 @@ import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestPrinter; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData; -import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.InternalConstants; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.CleanupJobStage; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.CreateOutputDirectoriesStage; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.LoadManifestsStage; @@ -39,6 +38,7 @@ import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.StageConfig; import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.DEFAULT_WRITER_QUEUE_CAPACITY; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_SUMMARY_REPORT_DIR; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_COMMIT; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.addHeapInformation; @@ -132,9 +132,10 @@ public void testSaveThenLoadManyManifests() throws Throwable { stageConfig); entryFile = File.createTempFile("entry", ".seq"); LoadManifestsStage.Arguments args = new LoadManifestsStage.Arguments( - entryFile, false, InternalConstants.ENTRY_WRITER_QUEUE_CAPACITY); - LoadManifestsStage.Result result = stage.apply(args); - LoadManifestsStage.SummaryInfo summary = result.getSummary(); + entryFile, false, DEFAULT_WRITER_QUEUE_CAPACITY); + + LoadManifestsStage.Result loadManifestsResult = stage.apply(args); + LoadManifestsStage.SummaryInfo summary = loadManifestsResult.getSummary(); LOG.info("\nJob statistics after loading {}", ioStatisticsToPrettyString(getStageStatistics())); @@ -162,7 +163,7 @@ public void testSaveThenLoadManyManifests() throws Throwable { // now let's see about aggregating a large set of directories Set createdDirectories = new CreateOutputDirectoriesStage( stageConfig) - .apply(result.getLoadedManifestData().getDirectories()) + .apply(loadManifestsResult.getLoadedManifestData().getDirectories()) .getCreatedDirectories(); addHeapInformation(heapInfo, "create.directories"); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/TestEntryFileIO.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/TestEntryFileIO.java index 4fd9fc0d8f6f8..ea3f1dcd750fd 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/TestEntryFileIO.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/TestEntryFileIO.java @@ -36,6 +36,7 @@ import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry; import org.apache.hadoop.test.AbstractHadoopTestBase; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.apache.hadoop.util.functional.RemoteIterators.foreach; /** @@ -88,7 +89,8 @@ private void setEntryFile(File entryFile) { /** - * Create a file with one entry + * Create a file with one entry, then read it back + * via all the mechanisms available. */ @Test public void testCreateWriteReadFileOneEntry() throws Throwable { @@ -115,6 +117,7 @@ public void testCreateWriteReadFileOneEntry() throws Throwable { List files = new ArrayList<>(); foreach(it, files::add); Assertions.assertThat(files) + .describedAs("iteration over the entry file") .hasSize(1) .element(0) .isEqualTo(source); @@ -126,14 +129,29 @@ public void testCreateWriteReadFileOneEntry() throws Throwable { .isEqualTo(1); } + /** + * Create a writer. + * @return a writer + * @throws IOException failure to create the file. + */ private SequenceFile.Writer createWriter() throws IOException { return entryFileIO.createWriter(getEntryFile()); } + /** + * Create an iterator over the records in the (non empty) entry file. + * @return an iterator over entries. + * @throws IOException failure to open the file + */ private RemoteIterator iterateOverEntryFile() throws IOException { return entryFileIO.iterateOver(readEntryFile()); } + /** + * Create a reader for the (non empty) entry file. + * @return a reader. + * @throws IOException failure to open the file + */ private SequenceFile.Reader readEntryFile() throws IOException { assertEntryFileNonEmpty(); @@ -163,7 +181,7 @@ private void assertEntryFileNonEmpty() { } /** - * Generate lots of data and write. + * Generate lots of data and write it. */ @Test public void testLargeStreamingWrite() throws Throwable { @@ -199,6 +217,8 @@ public void testLargeStreamingWrite() throws Throwable { .describedAs("total elements written") .isEqualTo(total); } + + // now read it back AtomicInteger count = new AtomicInteger(); foreach(iterateOverEntryFile(), e -> { final int elt = count.getAndIncrement(); @@ -210,8 +230,17 @@ public void testLargeStreamingWrite() throws Throwable { Assertions.assertThat(count.get()) .describedAs("total elements read") .isEqualTo(total); + } - + @Test + public void testCreateInvalidWriter() throws Throwable { + intercept(NullPointerException.class, () -> + entryFileIO.launchEntryWriter(null, 1)); + } + @Test + public void testCreateInvalidWriterCapacity() throws Throwable { + intercept(IllegalStateException.class, () -> + entryFileIO.launchEntryWriter(null, 0)); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java index da12223570ec8..8d92a7652d47e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java @@ -37,7 +37,7 @@ import org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum; import org.apache.hadoop.fs.s3a.statistics.impl.AbstractS3AStatisticsSource; import org.apache.hadoop.fs.s3a.statistics.impl.CountingChangeTracker; -import org.apache.hadoop.fs.s3a.statistics.impl.ForwardingIOStatisticsStore; +import org.apache.hadoop.fs.statistics.impl.ForwardingIOStatisticsStore; import org.apache.hadoop.fs.statistics.DurationTrackerFactory; import org.apache.hadoop.fs.statistics.IOStatisticsLogging; import org.apache.hadoop.fs.statistics.IOStatisticsSource; From ed04b54f6d3c4a55ed9665a5ddc8ec0663709bab Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 18 Apr 2023 21:17:48 +0100 Subject: [PATCH 09/18] MAPREDUCE-7435. Parallel writing test Change-Id: I2008d31bff3af59396a04dddc1b9357b1a812294 --- .../committer/manifest/impl/EntryFileIO.java | 91 +++++--- .../impl/ManifestCommitterSupport.java | 7 +- .../manifest/TestLoadManifestsStage.java | 19 +- .../manifest/impl/TestEntryFileIO.java | 206 ++++++++++++++++-- 4 files changed, 269 insertions(+), 54 deletions(-) diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java index f3d3c114f405b..32bd5e72c2064 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java @@ -22,7 +22,6 @@ import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; -import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.NoSuchElementException; @@ -177,6 +176,35 @@ public static Path toPath(final File file) { return new Path(file.toURI()); } + + /** + * Actions in the queue. + */ + private enum Actions { + write, + stop + } + + + /** + * What gets queued: an action and a list of entries + */ + private static final class QueueEntry { + + final Actions action; + + final List entries; + + private QueueEntry(final Actions action, List entries) { + this.action = action; + this.entries = entries; + } + + private QueueEntry(final Actions action) { + this(action, null); + } + } + /** * A Writer thread takes reads from a queue containing * list of entries to save; these are serialized via the writer to @@ -185,17 +213,20 @@ public static Path toPath(final File file) { * for them to be written. * The these threads will be blocked when the queue capacity is reached. */ - public final class EntryWriter implements Closeable { + public static final class EntryWriter implements Closeable { private final SequenceFile.Writer writer; - private final BlockingQueue> queue; + private final BlockingQueue queue; /** * stop flag. */ private final AtomicBoolean stop = new AtomicBoolean(false); + /** + * Is the processor thread active. + */ private final AtomicBoolean active = new AtomicBoolean(false); /** @@ -280,7 +311,7 @@ public boolean enqueue(List entries) { } if (active.get()) { try { - queue.put(entries); + queue.put(new QueueEntry(Actions.write, entries)); LOG.debug("Queued {}", entries.size()); return true; } catch (InterruptedException e) { @@ -299,25 +330,36 @@ public boolean enqueue(List entries) { * @throws UncheckedIOException on write failure */ private int processor() { + Thread.currentThread().setName("EntryIOWriter"); try { while (!stop.get()) { - final List entries = queue.take(); - if (entries.isEmpty()) { - // exit list reached. + final QueueEntry queueEntry = queue.take(); + switch (queueEntry.action) { + case stop: LOG.debug("List termination initiated"); stop.set(true); - } else { + break; + case write: + final List entries = queueEntry.entries; LOG.debug("Adding block of {} entries", entries.size()); for (FileEntry entry : entries) { append(entry); } + break; } } + } catch (IOException e) { + LOG.debug("Write failure", e); + failure.set(e); + throw new UncheckedIOException(e); } catch (InterruptedException e) { - // assume we are stopped here + // being stopped implicitly LOG.debug("interrupted", e); - active.set(false); + } finally { stop.set(true); + active.set(false); + // clear the queue, so wake up on any failure mode. + queue.clear(); } return count.get(); } @@ -325,27 +367,20 @@ private int processor() { /** * write one entry. * @param entry entry to write - * @throws UncheckedIOException on write failure + * @throws IOException on write failure */ - private void append(FileEntry entry) { - try { - writer.append(NullWritable.get(), entry); + private void append(FileEntry entry) throws IOException { + writer.append(NullWritable.get(), entry); - final int c = count.incrementAndGet(); - LOG.trace("Added entry #{}: {}", c, entry); - } catch (IOException e) { - LOG.debug("Write failure", e); - failure.set(e); - throw new UncheckedIOException(e); - } + final int c = count.incrementAndGet(); + LOG.trace("Added entry #{}: {}", c, entry); } /** - * Close: stop accepting new writes, wait for queued writes to complete + * Close: stop accepting new writes, wait for queued writes to complete. * @throws IOException failure closing that writer, or somehow the future * raises an IOE which isn't caught for later. */ - @Override public void close() throws IOException { @@ -357,18 +392,21 @@ public void close() throws IOException { return; } LOG.debug("Shutting down writer"); - // signal queue closure by queue an empty list + // signal queue closure by queuing a stop option. + // this is added at the end of the list of queued blocks, + // of which are written. try { - queue.put(new ArrayList<>()); + queue.put(new QueueEntry(Actions.stop)); } catch (InterruptedException e) { Thread.interrupted(); } try { // wait for the op to finish. - int total = FutureIO.awaitFuture(future, 1, TimeUnit.MINUTES); + int total = FutureIO.awaitFuture(future, 30, TimeUnit.SECONDS); LOG.debug("Processed {} files", total); executor.shutdown(); } catch (TimeoutException e) { + LOG.warn("Timeout waiting for write thread to finish"); // trouble. force close executor.shutdownNow(); // close the stream @@ -400,6 +438,7 @@ public String toString() { } } + /** * Iterator to retrieve file entries from the sequence file. * Closeable; it will close automatically when the last element is read. diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestCommitterSupport.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestCommitterSupport.java index c8ee77c84586d..ccbcfcba22ebf 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestCommitterSupport.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestCommitterSupport.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.time.ZonedDateTime; -import java.util.Map; import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.classification.InterfaceAudience; @@ -55,8 +54,6 @@ import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.JOB_TASK_MANIFEST_SUBDIR; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.MANIFEST_COMMITTER_CLASSNAME; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.MANIFEST_SUFFIX; -import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_IO_PROCESSORS; -import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_IO_PROCESSORS_DEFAULT; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_STORE_OPERATIONS_CLASS; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SPARK_WRITE_UUID; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SUMMARY_FILENAME_FORMAT; @@ -235,12 +232,10 @@ public static ManifestSuccessData createManifestOutcome( */ public static void addHeapInformation(IOStatisticsSetters ioStatisticsSetters, String stage) { - // force a gc. bit of bad form but it makes for better numbers - System.gc(); final long totalMemory = Runtime.getRuntime().totalMemory(); + final long freeMemory = Runtime.getRuntime().freeMemory(); final String prefix = "stage."; ioStatisticsSetters.setGauge(prefix + stage + "." + TOTAL_MEMORY, totalMemory); - final long freeMemory = Runtime.getRuntime().freeMemory(); ioStatisticsSetters.setGauge(prefix + stage + "." + FREE_MEMORY, freeMemory); ioStatisticsSetters.setGauge(prefix + stage + "." + HEAP_MEMORY, totalMemory - freeMemory); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestLoadManifestsStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestLoadManifestsStage.java index 8a7626116fa52..9b869d4c2e4cc 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestLoadManifestsStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestLoadManifestsStage.java @@ -123,7 +123,8 @@ public void testSaveThenLoadManyManifests() throws Throwable { executeTaskAttempts(taskAttemptCount, filesPerTaskAttempt); IOStatisticsSnapshot heapInfo = new IOStatisticsSnapshot(); - addHeapInformation(heapInfo, "initial"); + + heapinfo(heapInfo, "initial"); LOG.info("Loading in the manifests"); @@ -140,7 +141,7 @@ public void testSaveThenLoadManyManifests() throws Throwable { LOG.info("\nJob statistics after loading {}", ioStatisticsToPrettyString(getStageStatistics())); LOG.info("Heap size = {}", heapSize()); - addHeapInformation(heapInfo, "load.manifests"); + heapinfo(heapInfo, "load.manifests"); Assertions.assertThat(summary.getManifestCount()) @@ -165,7 +166,7 @@ public void testSaveThenLoadManyManifests() throws Throwable { stageConfig) .apply(loadManifestsResult.getLoadedManifestData().getDirectories()) .getCreatedDirectories(); - addHeapInformation(heapInfo, "create.directories"); + heapinfo(heapInfo, "create.directories"); // but after the merge process, only one per generated file output // dir exists @@ -177,7 +178,7 @@ public void testSaveThenLoadManyManifests() throws Throwable { // go straight to cleanup new CleanupJobStage(stageConfig).apply( new CleanupJobStage.Arguments("", true, true, false)); - addHeapInformation(heapInfo, "cleanup"); + heapinfo(heapInfo, "cleanup"); ManifestSuccessData success = createManifestOutcome(stageConfig, OP_STAGE_JOB_COMMIT); success.snapshotIOStatistics(getStageStatistics()); @@ -197,4 +198,14 @@ public void testSaveThenLoadManyManifests() throws Throwable { showManifest.loadAndPrintManifest(summaryFS, path); } + /** + * Force a GC then add heap info. + * @param stats stats to update + * @param stage stage name + */ + private static void heapinfo(final IOStatisticsSnapshot stats, final String stage) { + System.gc(); + addHeapInformation(stats, stage); + } + } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/TestEntryFileIO.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/TestEntryFileIO.java index ea3f1dcd750fd..d5ba98a498f5d 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/TestEntryFileIO.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/TestEntryFileIO.java @@ -22,19 +22,27 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.NoSuchElementException; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Before; import org.junit.Test; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.AbstractManifestCommitterTest; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry; -import org.apache.hadoop.test.AbstractHadoopTestBase; +import org.apache.hadoop.util.functional.TaskPool; import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.apache.hadoop.util.functional.RemoteIterators.foreach; @@ -42,8 +50,14 @@ /** * Test {@link EntryFileIO}. */ -public class TestEntryFileIO extends AbstractHadoopTestBase { +public class TestEntryFileIO extends AbstractManifestCommitterTest { + private static final Logger LOG = LoggerFactory.getLogger( + TestEntryFileIO.class); + + /** + * Entry to save. + */ public static final FileEntry ENTRY = new FileEntry("source", "dest", 100, "etag"); /** @@ -51,8 +65,14 @@ public class TestEntryFileIO extends AbstractHadoopTestBase { */ private EntryFileIO entryFileIO; + /** + * Path to a test entry file. + */ private File entryFile; + /** + * Create an entry file during setup. + */ @Before public void setup() throws Exception { entryFileIO = new EntryFileIO(new Configuration()); @@ -60,7 +80,7 @@ public void setup() throws Exception { } /** - * Teardown. + * Teardown deletes any entry file. * @throws Exception on any failure */ @After @@ -71,7 +91,10 @@ public void teardown() throws Exception { } } - + /** + * Create a temp entry file and set the entryFile field to it. + * @throws IOException creation failure + */ private void createEntryFile() throws IOException { setEntryFile(File.createTempFile("entry", ".seq")); } @@ -87,7 +110,6 @@ private void setEntryFile(File entryFile) { this.entryFile = entryFile; } - /** * Create a file with one entry, then read it back * via all the mechanisms available. @@ -159,7 +181,7 @@ private SequenceFile.Reader readEntryFile() throws IOException { } /** - * Create a file with one entry + * Create a file with one entry. */ @Test public void testCreateEmptyFile() throws Throwable { @@ -180,6 +202,19 @@ private void assertEntryFileNonEmpty() { .isGreaterThan(0); } + @Test + public void testCreateInvalidWriter() throws Throwable { + intercept(NullPointerException.class, () -> + entryFileIO.launchEntryWriter(null, 1)); + } + + @Test + public void testCreateInvalidWriterCapacity() throws Throwable { + intercept(IllegalStateException.class, () -> + entryFileIO.launchEntryWriter(null, 0)); + } + + /** * Generate lots of data and write it. */ @@ -190,12 +225,8 @@ public void testLargeStreamingWrite() throws Throwable { int listSize = 100; // and the number of block writes int writes = 100; - List list = new ArrayList<>(listSize); - for (int i = 0; i < listSize; i++) { - list.add(new FileEntry("source" + i, "dest" + i, i, "etag-" + i)); - } - // just for debugging/regression testing - Assertions.assertThat(list).hasSize(listSize); + List list = buildEntryList(listSize); + int total = listSize * writes; try (EntryFileIO.EntryWriter out = entryFileIO.launchEntryWriter(createWriter(), 2)) { @@ -232,15 +263,154 @@ public void testLargeStreamingWrite() throws Throwable { .isEqualTo(total); } + /** + * Build an entry list. + * @param listSize size of the list + * @return a list of entries + */ + private static List buildEntryList(final int listSize) { + List list = new ArrayList<>(listSize); + for (int i = 0; i < listSize; i++) { + list.add(new FileEntry("source" + i, "dest" + i, i, "etag-" + i)); + } + // just for debugging/regression testing + Assertions.assertThat(list).hasSize(listSize); + return list; + } + + /** + * Write lists to the output, but the stream is going to fail after a + * configured number of records have been written. + * Verify that the (blocked) submitter is woken up + * and that the exception was preserved for rethrowing. + */ @Test - public void testCreateInvalidWriter() throws Throwable { - intercept(NullPointerException.class, () -> - entryFileIO.launchEntryWriter(null, 1)); + public void testFailurePropagation() throws Throwable { + + final int count = 4; + final SequenceFile.Writer writer = spyWithFailingAppend( + entryFileIO.createWriter(getEntryFile()), count); + // list of 100 entries at a time + // and the number of block writes + List list = buildEntryList(1); + + // small queue ensures the posting thread is blocked + try (EntryFileIO.EntryWriter out = entryFileIO.launchEntryWriter(writer, 2)) { + boolean valid = true; + for (int i = 0; valid && i < count * 2; i++) { + valid = out.enqueue(list); + } + LOG.info("queue to {} finished valid={}", out, valid); + out.close(); + + // verify the exception is as expected + intercept(IOException.class, "mocked", () -> + out.maybeRaiseWriteException()); + + // and verify the count of invocations. + Assertions.assertThat(out.getCount()) + .describedAs("process count of %s", count) + .isEqualTo(count); + } + } + + /** + * Spy on a writer with the append operation to fail after the given count of calls + * is reached. + * @param writer write. + * @param count number of allowed append calls. + * @return spied writer. + * @throws IOException from the signature of the append() call mocked. + */ + private static SequenceFile.Writer spyWithFailingAppend(final SequenceFile.Writer writer, + final int count) + throws IOException { + AtomicLong limit = new AtomicLong(count); + + final SequenceFile.Writer spied = Mockito.spy(writer); + Mockito.doAnswer((InvocationOnMock invocation) -> { + final Writable k = invocation.getArgument(0); + final Writable v = invocation.getArgument(1); + if (limit.getAndDecrement() > 0) { + writer.append(k, v); + } else { + throw new IOException("mocked"); + } + return null; + }).when(spied).append(Mockito.any(Writable.class), Mockito.any(Writable.class)); + return spied; } + + + /** + * Multithreaded writing. + */ @Test - public void testCreateInvalidWriterCapacity() throws Throwable { - intercept(IllegalStateException.class, () -> - entryFileIO.launchEntryWriter(null, 0)); + public void testParallelWrite() throws Throwable { + + // list of 100 entries at a time + int listSize = 100; + // and the number of block writes + int attempts = 100; + List list = buildEntryList(listSize); + + int total = listSize * attempts; + + + try (EntryFileIO.EntryWriter out = entryFileIO.launchEntryWriter(createWriter(), 20)) { + + + TaskPool.foreach(new LongIterator(0, attempts)) + .executeWith(getSubmitter()) + .stopOnFailure() + .run(l -> {out.enqueue(list);}); + out.close(); + out.maybeRaiseWriteException(); + + + Assertions.assertThat(out.getCount()) + .describedAs("total elements written") + .isEqualTo(total); + } + + // now read it back + AtomicInteger count = new AtomicInteger(); + foreach(iterateOverEntryFile(), e -> { + final int elt = count.getAndIncrement(); + }); + Assertions.assertThat(count.get()) + .describedAs("total elements read") + .isEqualTo(total); } + + /** + * To allow us to use TaskPool. + */ + private static final class LongIterator implements RemoteIterator { + + private long start; + + private final long finish; + + private LongIterator(final long start, final long finish) { + this.start = start; + this.finish = finish; + } + + @Override + public boolean hasNext() throws IOException { + return start < finish; + } + + @Override + public Long next() throws IOException { + if (!hasNext()) { + throw new NoSuchElementException(); + } + final long s = start; + start++; + return s; + } + } } From 61a684614a81eae8ab43d9038fdda3650632beb4 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 19 Apr 2023 16:32:36 +0100 Subject: [PATCH 10/18] MAPREDUCE-7435. checkstyle, remote iterator work, azure tuning * moved RangeExcludingLongIterator into RemoteIterators, added test. * address checkstyle * address spotbugs * address deprecation * ValidateRenameFilesStage doesn't validate etags on wasb; helps address a JIRA about hadoop-azure testing. Change-Id: Id6507d79f8d3cfa434afb65bfe9fc7539a7c1cf5 --- .../util/functional/RemoteIterators.java | 53 +++++++++++++++++++ .../util/functional/TestRemoteIterators.java | 43 ++++++++++----- .../committer/manifest/impl/EntryFileIO.java | 20 +++++-- .../manifest/impl/LoadedManifestData.java | 9 ++-- .../stages/AbstractJobOrTaskStage.java | 11 +++- .../manifest/stages/CommitJobStage.java | 5 +- .../manifest/stages/LoadManifestsStage.java | 35 ++++++------ .../stages/ValidateRenamedFilesStage.java | 3 +- .../src/site/markdown/manifest_committer.md | 2 +- .../TestJobThroughManifestCommitter.java | 2 +- .../manifest/impl/TestEntryFileIO.java | 47 +++------------- .../commit/AbfsManifestStoreOperations.java | 14 ++++- 12 files changed, 158 insertions(+), 86 deletions(-) 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 index 3140e5eb74d80..30605e1a09469 100644 --- 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 @@ -205,6 +205,16 @@ public static RemoteIterator haltableRemoteIterator( return new HaltableRemoteIterator<>(iterator, continueWork); } + /** + * A remote iterator which simply counts up, stopping once the + * value is greater than the finish. + * This is primarily for tests or when submitting work into a TaskPool. + */ + public static RemoteIterator rangeExcludingIterator( + final long start, final long finish) { + return new RangeExcludingLongIterator(start, finish); + } + /** * Build a list from a RemoteIterator. * @param source source iterator @@ -776,4 +786,47 @@ protected boolean sourceHasNext() throws IOException { } } + /** + * A remote iterator which simply counts up, stopping once the + * value is greater than the finish. + * This is primarily for tests or when submitting work into a TaskPool. + */ + private static final class RangeExcludingLongIterator implements RemoteIterator { + + /** + * Current value. + */ + private long current; + + /** + * End value. + */ + private final long finish; + + /** + * Construct. + * @param start start value. + * @param finish halt the iterator once the current value is equal to or greater than this. + */ + private RangeExcludingLongIterator(final long start, final long finish) { + this.current = start; + this.finish = finish; + } + + @Override + public boolean hasNext() throws IOException { + return current < finish; + } + + @Override + public Long next() throws IOException { + if (!hasNext()) { + throw new NoSuchElementException(); + } + final long s = current; + current++; + return s; + } + } + } 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 index a95360b9a0d8d..2ff730242a404 100644 --- 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 @@ -26,7 +26,6 @@ import org.apache.hadoop.util.Preconditions; -import org.assertj.core.api.Assertions; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -298,30 +297,35 @@ public void testHaltableIterator() throws Throwable { // a countdown of 10, but the halting predicate will fail earlier // if the value of "count" has dropped to zero - final RemoteIterator it = + final RemoteIterator it = haltableRemoteIterator( - new CountdownRemoteIterator(10), + rangeExcludingIterator(0,10), () -> count.get() > 0); - Assertions.assertThat(foreach(it, - (v) -> count.decrementAndGet())) - .describedAs("Count of iterations") - .isEqualTo(limit); + verifyInvoked(it, limit, (v) -> count.decrementAndGet()); } + @Test public void testHaltableIteratorNoHalt() throws Throwable { // a countdown of 10, but the halting predicate will fail earlier // if the value of "count" has dropped to zero - final RemoteIterator it = + final int finish = 10; + final RemoteIterator it = haltableRemoteIterator( - new CountdownRemoteIterator(10), + rangeExcludingIterator(0, finish), () -> true); - Assertions.assertThat(foreach(it, - (v) -> {})) - .describedAs("Count of iterations") - .isEqualTo(10); + verifyInvoked(it, finish); + } + + @Test + public void testRangeExcludingIterator() throws Throwable { + verifyInvoked(rangeExcludingIterator(0, 0), 0); + verifyInvoked(rangeExcludingIterator(0, -1), 0); + verifyInvoked(rangeExcludingIterator(0, 100), 100); + intercept(NoSuchElementException.class, () -> + rangeExcludingIterator(0, 0).next()); } /** @@ -364,6 +368,19 @@ protected void verifyInvoked(final RemoteIterator it, .isEqualTo(length); } + /** + * Verify that the iteration completes with a given invocation count. + * @param it iterator + * @param type. + * @param length expected size + */ + protected void verifyInvoked( + final RemoteIterator it, + final int length) + throws IOException { + verifyInvoked(it, length, (t) -> { }); + + } /** * Close an iterator if it is iterable. * @param it iterator diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java index 32bd5e72c2064..32f5ba85ea096 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java @@ -181,19 +181,20 @@ public static Path toPath(final File file) { * Actions in the queue. */ private enum Actions { + /** Write the supplied list of entries. */ write, + /** Stop the processor thread. */ stop } - /** - * What gets queued: an action and a list of entries + * What gets queued: an action and a list of entries. */ private static final class QueueEntry { - final Actions action; + private final Actions action; - final List entries; + private final List entries; private QueueEntry(final Actions action, List entries) { this.action = action; @@ -212,11 +213,21 @@ private QueueEntry(final Actions action) { * Other threads can queue the file entry lists from loaded manifests * for them to be written. * The these threads will be blocked when the queue capacity is reached. + * This is quite a complex process, with the main troublespots in the code + * being: + * - managing the shutdown + * - failing safely on write failures, restarting all blocked writers in the process */ public static final class EntryWriter implements Closeable { + /** + * The destination of the output. + */ private final SequenceFile.Writer writer; + /** + * Blocking queue of actions. + */ private final BlockingQueue queue; /** @@ -329,6 +340,7 @@ public boolean enqueue(List entries) { * @return count of entries written. * @throws UncheckedIOException on write failure */ + @SuppressWarnings("SwitchStatementWithoutDefaultBranch") private int processor() { Thread.currentThread().setName("EntryIOWriter"); try { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/LoadedManifestData.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/LoadedManifestData.java index d816c817599d9..4f3ec71216915 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/LoadedManifestData.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/LoadedManifestData.java @@ -52,7 +52,7 @@ public final class LoadedManifestData { private final int fileCount; /** - * Data about the loaded manifests + * Data about the loaded manifests. * @param directories directories * @param entrySequenceData Path in local fs to the entry sequence data. * @param fileCount number of files. @@ -83,7 +83,7 @@ public Path getEntrySequenceData() { } /** - * Get the entry sequence data as a file; + * Get the entry sequence data as a file. */ public File getEntrySequenceFile() { return new File(entrySequenceData.toUri()); @@ -91,8 +91,9 @@ public File getEntrySequenceFile() { /** * Delete the entry sequence file. + * @return whether or not the delete was successful. */ - public void deleteEntrySequenceFile() { - getEntrySequenceFile().delete(); + public boolean deleteEntrySequenceFile() { + return getEntrySequenceFile().delete(); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/AbstractJobOrTaskStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/AbstractJobOrTaskStage.java index 05ee7a5ac11d7..3b4a21e387946 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/AbstractJobOrTaskStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/AbstractJobOrTaskStage.java @@ -161,7 +161,7 @@ protected AbstractJobOrTaskStage( getRequiredTaskAttemptId(); getRequiredTaskAttemptDir(); stageName = String.format("[Task-Attempt %s]", getRequiredTaskAttemptId()); - } else { + } else { stageName = String.format("[Job-Attempt %s/%02d]", stageConfig.getJobId(), stageConfig.getJobAttemptNumber()); @@ -312,6 +312,15 @@ private void noteAnyRateLimiting(String statistic, Duration wait) { } } + + /** + * Get the operations callbacks. + * @return the operations invocable against the destination. + */ + public ManifestStoreOperations getOperations() { + return operations; + } + @Override public String toString() { final StringBuilder sb = new StringBuilder( diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java index 02a9968407d29..c5ab7f1551523 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java @@ -34,7 +34,6 @@ import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.LoadedManifestData; import static java.util.Objects.requireNonNull; -import static org.apache.commons.io.FileUtils.byteCountToDisplaySize; import static org.apache.commons.lang3.StringUtils.isNotBlank; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_BYTES_COMMITTED_COUNT; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_FILES_COMMITTED_COUNT; @@ -94,7 +93,7 @@ protected CommitJobStage.Result executeStage( LOG.info("{}: Committing job with file count: {}; total size {} bytes", getName(), summary.getFileCount(), - byteCountToDisplaySize(summary.getTotalFileSize())); + String.format("%,d", summary.getTotalFileSize())); addHeapInformation(heapInfo, OP_STAGE_JOB_LOAD_MANIFESTS); @@ -178,7 +177,7 @@ protected CommitJobStage.Result executeStage( // the result return new Result(successPath, successData); } finally { - // cleanup + // cleanup; return code is ignored. if (loadedManifestData != null) { loadedManifestData.deleteEntrySequenceFile(); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java index 7c8e84a5a2319..1874c82232758 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java @@ -24,6 +24,7 @@ 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; @@ -363,6 +364,8 @@ public LoadedManifestData getLoadedManifestData() { /** * 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 { @@ -379,12 +382,12 @@ public static final class SummaryInfo implements IOStatisticsSource { /** * How many manifests were loaded. */ - private long manifestCount; + private AtomicLong manifestCount = new AtomicLong(); /** * Total number of files to rename. */ - private long fileCount; + private AtomicLong fileCount = new AtomicLong(); /** * Total number of directories which may need @@ -392,12 +395,12 @@ public static final class SummaryInfo implements IOStatisticsSource { * As there is no dedup, this is likely to be * a (major) overestimate. */ - private long directoryCount; + private AtomicLong directoryCount = new AtomicLong(); /** * Total amount of data to be committed. */ - private long totalFileSize; + private AtomicLong totalFileSize = new AtomicLong(); /** * Get the IOStatistics. @@ -409,19 +412,19 @@ public IOStatisticsSnapshot getIOStatistics() { } public long getFileCount() { - return fileCount; + return fileCount.get(); } public long getDirectoryCount() { - return directoryCount; + return directoryCount.get(); } public long getTotalFileSize() { - return totalFileSize; + return totalFileSize.get(); } public long getManifestCount() { - return manifestCount; + return manifestCount.get(); } public List getTaskIDs() { @@ -433,11 +436,11 @@ public List getTaskIDs() { * @param manifest manifest to add. */ public synchronized void add(TaskManifest manifest) { - manifestCount++; + manifestCount.incrementAndGet(); 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()); } @@ -449,11 +452,11 @@ public synchronized void add(TaskManifest manifest) { public String toString() { final StringBuilder sb = new StringBuilder( "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( - byteCountToDisplaySize(totalFileSize)); + byteCountToDisplaySize(getTotalFileSize())); sb.append('}'); return sb.toString(); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/ValidateRenamedFilesStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/ValidateRenamedFilesStage.java index 37c44a71f2972..070328ab414f4 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/ValidateRenamedFilesStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/ValidateRenamedFilesStage.java @@ -139,7 +139,8 @@ private void validateOneFile(FileEntry entry) throws IOException { // etags, if the source had one. final String sourceEtag = entry.getEtag(); - if (isNotBlank(sourceEtag)) { + if (getOperations().storePreservesEtagsThroughRenames(destStatus.getPath()) + && isNotBlank(sourceEtag)) { final String destEtag = ManifestCommitterSupport.getEtag(destStatus); if (!sourceEtag.equals(destEtag)) { LOG.warn("Etag of dest file {}: {} does not match that of manifest entry {}", diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer.md b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer.md index bf37feff408ec..3765efcd56d85 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer.md +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer.md @@ -243,7 +243,7 @@ Caveats This is a secondary scale option. It controls the size of the queue for storing lists of files to rename from -the manifests loaded from the target filesystem, manifests loaded +the manifests loaded from the target filesystem, manifests loaded from a pool of worker threads, and the single thread which saves the entries from each manifest to an intermediate file in the local filesystem. diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java index feeb20d348108..08011a20c5aa2 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java @@ -514,7 +514,7 @@ public void test_0420_validateJob() throws Throwable { .containsAll(successData.getFilenames()); // delete an entry, repeat - getFileSystem().delete(validatedEntries.get(0).getDestPath()); + getFileSystem().delete(validatedEntries.get(0).getDestPath(), false); intercept(OutputValidationException.class, () -> new ValidateRenamedFilesStage(getJobStageConfig()) .apply(loadedManifestData.getEntrySequenceData())); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/TestEntryFileIO.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/TestEntryFileIO.java index d5ba98a498f5d..dff7acee7be32 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/TestEntryFileIO.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/TestEntryFileIO.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.NoSuchElementException; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -46,6 +45,7 @@ import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.apache.hadoop.util.functional.RemoteIterators.foreach; +import static org.apache.hadoop.util.functional.RemoteIterators.rangeExcludingIterator; /** * Test {@link EntryFileIO}. @@ -358,59 +358,24 @@ public void testParallelWrite() throws Throwable { try (EntryFileIO.EntryWriter out = entryFileIO.launchEntryWriter(createWriter(), 20)) { - - - TaskPool.foreach(new LongIterator(0, attempts)) + TaskPool.foreach(rangeExcludingIterator(0, attempts)) .executeWith(getSubmitter()) .stopOnFailure() - .run(l -> {out.enqueue(list);}); + .run(l -> { + out.enqueue(list); + }); out.close(); out.maybeRaiseWriteException(); - Assertions.assertThat(out.getCount()) .describedAs("total elements written") .isEqualTo(total); } // now read it back - AtomicInteger count = new AtomicInteger(); - foreach(iterateOverEntryFile(), e -> { - final int elt = count.getAndIncrement(); - }); - Assertions.assertThat(count.get()) + Assertions.assertThat(foreach(iterateOverEntryFile(), e -> { })) .describedAs("total elements read") .isEqualTo(total); } - - /** - * To allow us to use TaskPool. - */ - private static final class LongIterator implements RemoteIterator { - - private long start; - - private final long finish; - - private LongIterator(final long start, final long finish) { - this.start = start; - this.finish = finish; - } - - @Override - public boolean hasNext() throws IOException { - return start < finish; - } - - @Override - public Long next() throws IOException { - if (!hasNext()) { - throw new NoSuchElementException(); - } - final long s = start; - start++; - return s; - } - } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/commit/AbfsManifestStoreOperations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/commit/AbfsManifestStoreOperations.java index efba9244af700..6bfab3a8515a9 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/commit/AbfsManifestStoreOperations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/commit/AbfsManifestStoreOperations.java @@ -62,6 +62,11 @@ public class AbfsManifestStoreOperations extends */ private ResilientCommitByRename resilientCommitByRename; + /** + * Are etags preserved in renames? + */ + private boolean etagsPreserved; + @Override public AzureBlobFileSystem getFileSystem() { return (AzureBlobFileSystem) super.getFileSystem(); @@ -83,15 +88,22 @@ public void bindToFileSystem(FileSystem filesystem, Path path) throws IOExceptio super.bindToFileSystem(filesystem, path); try { resilientCommitByRename = getFileSystem().createResilientCommitSupport(path); + // this also means that etags are preserved. + etagsPreserved = true; LOG.debug("Bonded to filesystem with resilient commits under path {}", path); } catch (UnsupportedOperationException e) { LOG.debug("No resilient commit support under path {}", path); } } + /** + * Etags are preserved through Gen2 stores, but not wasb stores. + * @param path path to probe. + * @return true if this store preserves etags. + */ @Override public boolean storePreservesEtagsThroughRenames(final Path path) { - return true; + return etagsPreserved; } /** From ffb25e7c5151f0c2946e48f268d4704531c6b55b Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 19 Apr 2023 19:47:43 +0100 Subject: [PATCH 11/18] MAPREDUCE-7435. improve ITestAbfsLoadManifestsStage performance * back to the original 200 manifest files * increase worker pool and buffer queue size (more significant before reducing the manifest count) brings test time down to 10s locally. IOStats does imply many MB of data is being PUT/GET so it is good to keep small so people running with less bandwidth don't suffer. Maybe, maybe, the size could switch with a -Dscale? Change-Id: I49d201d7af7434797ab6fff5831a0f899c5c4185 --- .../committer/manifest/ManifestCommitterTestSupport.java | 7 +++++++ .../fs/azurebfs/commit/ITestAbfsLoadManifestsStage.java | 8 ++++++++ 2 files changed, 15 insertions(+) diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterTestSupport.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterTestSupport.java index 20bd6e93baa15..a398bf1da58eb 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterTestSupport.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterTestSupport.java @@ -83,6 +83,13 @@ public final class ManifestCommitterTestSupport { */ public static final int NUMBER_OF_TASK_ATTEMPTS = 2000; + /** + * Smaller number of task attempts for some tests against object + * stores where IO overhead is higher. + * Value: {@value}. + */ + public static final int NUMBER_OF_TASK_ATTEMPTS_SMALL = 200; + private ManifestCommitterTestSupport() { } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsLoadManifestsStage.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsLoadManifestsStage.java index 8efdc7fb6f687..367692fca5d22 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsLoadManifestsStage.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsLoadManifestsStage.java @@ -23,6 +23,7 @@ import org.apache.hadoop.fs.azurebfs.contract.ABFSContractTestBinding; import org.apache.hadoop.fs.azurebfs.contract.AbfsFileSystemContract; import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterTestSupport; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.TestLoadManifestsStage; /** @@ -58,4 +59,11 @@ protected int getTestTimeoutMillis() { return AzureTestConstants.SCALE_TEST_TIMEOUT_MILLIS; } + /** + * @return a smaller number of TAs than the base test suite does. + */ + @Override + protected int numberOfTaskAttempts() { + return ManifestCommitterTestSupport.NUMBER_OF_TASK_ATTEMPTS_SMALL; + } } From 9874acbf8c8c1eab21f7b9eba9a802660c124b8f Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 19 Apr 2023 19:56:37 +0100 Subject: [PATCH 12/18] MAPREDUCE-7435. tweak test performance by disabling parallel TA dir cleanup Change-Id: If043263676c4d5694065e7ec35954a7f66c04d90 --- .../hadoop/fs/azurebfs/commit/AbfsCommitTestHelper.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/AbfsCommitTestHelper.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/AbfsCommitTestHelper.java index 37dcfd7c45148..da2a650489077 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/AbfsCommitTestHelper.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/AbfsCommitTestHelper.java @@ -20,6 +20,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.azurebfs.contract.ABFSContractTestBinding; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_READ_SMALL_FILES_COMPLETELY; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_STORE_OPERATIONS_CLASS; @@ -46,6 +47,13 @@ static Configuration prepareTestConfiguration( AbfsManifestStoreOperations.NAME); // turn on small file read if not explicitly set to a value. conf.setBooleanIfUnset(AZURE_READ_SMALL_FILES_COMPLETELY, true); + // use a larger thread pool to compensate for latencies + final String size = Integer.toString(192); + conf.setIfUnset(ManifestCommitterConstants.OPT_IO_PROCESSORS, size); + conf.setIfUnset(ManifestCommitterConstants.OPT_WRITER_QUEUE_CAPACITY, size); + // no need for parallel delete here as we aren't at the scale where unified delete + // is going to time out + conf.setBooleanIfUnset(ManifestCommitterConstants.OPT_CLEANUP_PARALLEL_DELETE, false); return conf; } From 7af5ab2a857243307c8b8bd6aae8c42bf991ac26 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 19 Apr 2023 20:13:06 +0100 Subject: [PATCH 13/18] MAPREDUCE-7435. reduce delete overhead on renaming by a HEAD This optimization assumes that most rename operation dest files don't exist, so a lower cost HEAD is justified, even though when there is an overwrite it is now a HEAD + DELETE, so more expensive. Change-Id: I968ffd897647b50d31a84da1268b69dc9a015924 --- .../manifest/ManifestCommitterConstants.java | 2 +- .../manifest/stages/AbstractJobOrTaskStage.java | 15 +++++++++++---- 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConstants.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConstants.java index d2a39a50cadb5..dc5ccb2e1df3a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConstants.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConstants.java @@ -258,7 +258,7 @@ public final class ManifestCommitterConstants { * Default value of {@link #OPT_WRITER_QUEUE_CAPACITY}. * Value {@value}. */ - public static final int DEFAULT_WRITER_QUEUE_CAPACITY = 32; + public static final int DEFAULT_WRITER_QUEUE_CAPACITY = OPT_IO_PROCESSORS_DEFAULT; private ManifestCommitterConstants() { } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/AbstractJobOrTaskStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/AbstractJobOrTaskStage.java index 3b4a21e387946..161153c82faac 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/AbstractJobOrTaskStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/AbstractJobOrTaskStage.java @@ -686,11 +686,18 @@ protected boolean storeSupportsResilientCommit() { return operations.storeSupportsResilientCommit(); } + /** + * Maybe delete the destination. + * This routine is optimized for the data not existing, as HEAD seems to cost less + * than a DELETE; assuming most calls don't have data, this is faster. + * @param deleteDest should an attempt to delete the dest be made? + * @param dest destination path + * @throws IOException IO failure, including permissions. + */ private void maybeDeleteDest(final boolean deleteDest, final Path dest) throws IOException { - if (deleteDest) { - // delete the destination, always, knowing that it's a no-op if - // the data isn't there. Skipping the change saves one round trip - // to actually look for the file/object + + if (deleteDest && getFileStatusOrNull(dest) != null) { + boolean deleted = delete(dest, true); // log the outcome in case of emergency diagnostics traces // being needed. From f969993de59a2063ec16350c8324f52938b4209d Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 20 Apr 2023 11:57:07 +0100 Subject: [PATCH 14/18] MAPREDUCE-7435. javadocs and other warnings, *not spotbugs* Change-Id: Ifacee8f8d47fb38a7702e788fa56556d0d31de0d --- .../util/functional/RemoteIterators.java | 24 ++++++++++++------- .../util/functional/TestRemoteIterators.java | 2 +- .../committer/manifest/impl/EntryFileIO.java | 11 +++++---- 3 files changed, 24 insertions(+), 13 deletions(-) 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 index 30605e1a09469..8b300052ce6c9 100644 --- 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 @@ -207,12 +207,19 @@ public static RemoteIterator haltableRemoteIterator( /** * A remote iterator which simply counts up, stopping once the - * value is greater than the finish. + * value is greater than the value of {@code excludedFinish}. * This is primarily for tests or when submitting work into a TaskPool. + * equivalent to + *
+   *   for(long l = start, l < finis; l++) yield l;
+   * 
+ * @param start start value + * @param excludedFinish excluded finish + * @return an iterator which returns longs from [start, finish) */ public static RemoteIterator rangeExcludingIterator( - final long start, final long finish) { - return new RangeExcludingLongIterator(start, finish); + final long start, final long excludedFinish) { + return new RangeExcludingLongIterator(start, excludedFinish); } /** @@ -801,21 +808,22 @@ private static final class RangeExcludingLongIterator implements RemoteIterator< /** * End value. */ - private final long finish; + private final long excludedFinish; /** * Construct. * @param start start value. - * @param finish halt the iterator once the current value is equal to or greater than this. + * @param excludedFinish halt the iterator once the current value is equal + * to or greater than this. */ - private RangeExcludingLongIterator(final long start, final long finish) { + private RangeExcludingLongIterator(final long start, final long excludedFinish) { this.current = start; - this.finish = finish; + this.excludedFinish = excludedFinish; } @Override public boolean hasNext() throws IOException { - return current < finish; + return current < excludedFinish; } @Override 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 index 2ff730242a404..373e1003ef728 100644 --- 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 @@ -299,7 +299,7 @@ public void testHaltableIterator() throws Throwable { // if the value of "count" has dropped to zero final RemoteIterator it = haltableRemoteIterator( - rangeExcludingIterator(0,10), + rangeExcludingIterator(0, 10), () -> count.get() > 0); verifyInvoked(it, limit, (v) -> count.decrementAndGet()); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java index 32f5ba85ea096..08137a5b88951 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java @@ -340,18 +340,21 @@ public boolean enqueue(List entries) { * @return count of entries written. * @throws UncheckedIOException on write failure */ - @SuppressWarnings("SwitchStatementWithoutDefaultBranch") private int processor() { Thread.currentThread().setName("EntryIOWriter"); try { while (!stop.get()) { final QueueEntry queueEntry = queue.take(); switch (queueEntry.action) { - case stop: - LOG.debug("List termination initiated"); + + case stop: // stop the operation + LOG.debug("Stop processing"); stop.set(true); break; - case write: + + case write: // write data + default: // here to shut compiler up + // write final List entries = queueEntry.entries; LOG.debug("Adding block of {} entries", entries.size()); for (FileEntry entry : entries) { From 8e83fdc70fdc9c9bc4b7ca2ee7934b76aa4d5b17 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 26 Apr 2023 12:34:09 +0100 Subject: [PATCH 15/18] MAPREDUCE-7435. Mehakmeet comments, excluding timeouts. Change-Id: Ib93ba8ba632135a05da126a75f34e78bd381cf2a --- .../hadoop/util/functional/RemoteIterators.java | 6 +++--- .../committer/manifest/impl/EntryFileIO.java | 15 +++++++++++---- .../manifest/impl/InternalConstants.java | 1 - .../committer/manifest/stages/CommitJobStage.java | 2 +- .../stages/CreateOutputDirectoriesStage.java | 14 ++++++++++++-- .../src/site/markdown/manifest_committer.md | 2 +- 6 files changed, 28 insertions(+), 12 deletions(-) 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 index 8b300052ce6c9..9b1611d7dadb0 100644 --- 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 @@ -211,7 +211,7 @@ public static RemoteIterator haltableRemoteIterator( * This is primarily for tests or when submitting work into a TaskPool. * equivalent to *
-   *   for(long l = start, l < finis; l++) yield l;
+   *   for(long l = start, l < excludedFinish; l++) yield l;
    * 
* @param start start value * @param excludedFinish excluded finish @@ -422,8 +422,8 @@ public void close() throws IOException { /** * Wrapper of another remote iterator; IOStatistics * and Closeable methods are passed down if implemented. - * This class may be subclasses if custom iterators - * are needed. + * This class may be subclassed within the hadoop codebase + * if custom iterators are needed. * @param source type * @param type of returned value */ diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java index 08137a5b88951..93f73822b8c51 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java @@ -63,6 +63,8 @@ public class EntryFileIO { private static final Logger LOG = LoggerFactory.getLogger( EntryFileIO.class); + public static final int WRITER_SHUTDOWN_TIMEOUT = 60; + /** Configuration used to load filesystems. */ private final Configuration conf; @@ -212,7 +214,7 @@ private QueueEntry(final Actions action) { * the output stream. * Other threads can queue the file entry lists from loaded manifests * for them to be written. - * The these threads will be blocked when the queue capacity is reached. + * These threads will be blocked when the queue capacity is reached. * This is quite a complex process, with the main troublespots in the code * being: * - managing the shutdown @@ -240,6 +242,8 @@ public static final class EntryWriter implements Closeable { */ private final AtomicBoolean active = new AtomicBoolean(false); + private final int capacity; + /** * Executor of writes. */ @@ -271,6 +275,7 @@ public static final class EntryWriter implements Closeable { private EntryWriter(SequenceFile.Writer writer, int capacity) { checkState(capacity > 0, "invalid queue capacity %s", capacity); this.writer = requireNonNull(writer); + this.capacity = capacity; this.queue = new ArrayBlockingQueue<>(capacity); } @@ -330,7 +335,7 @@ public boolean enqueue(List entries) { return false; } } else { - LOG.debug("Queue inactive; discarding {} entries", entries.size()); + LOG.warn("EntryFile write queue inactive; discarding {} entries", entries.size()); return false; } } @@ -406,7 +411,9 @@ public void close() throws IOException { // already stopped return; } - LOG.debug("Shutting down writer"); + LOG.debug("Shutting down writer; entry lists in queue: {}", + capacity - queue.remainingCapacity()); + // signal queue closure by queuing a stop option. // this is added at the end of the list of queued blocks, // of which are written. @@ -417,7 +424,7 @@ public void close() throws IOException { } try { // wait for the op to finish. - int total = FutureIO.awaitFuture(future, 30, TimeUnit.SECONDS); + int total = FutureIO.awaitFuture(future, WRITER_SHUTDOWN_TIMEOUT, TimeUnit.SECONDS); LOG.debug("Processed {} files", total); executor.shutdown(); } catch (TimeoutException e) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/InternalConstants.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/InternalConstants.java index 9f4fcde10857c..7155432c04287 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/InternalConstants.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/InternalConstants.java @@ -40,7 +40,6 @@ */ @InterfaceAudience.Private public final class InternalConstants { - private InternalConstants() { } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java index c5ab7f1551523..35f0df2d08662 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java @@ -72,7 +72,7 @@ protected CommitJobStage.Result executeStage( storeSupportsResilientCommit()); // once the manifest has been loaded, a temp file needs to be - // deleted; so track teh value. + // deleted; so track the value. LoadedManifestData loadedManifestData = null; try { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CreateOutputDirectoriesStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CreateOutputDirectoriesStage.java index 1b717886c71de..1618cf591a590 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CreateOutputDirectoriesStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CreateOutputDirectoriesStage.java @@ -37,6 +37,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DirEntry; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.EntryStatus; import org.apache.hadoop.util.functional.TaskPool; @@ -242,6 +243,7 @@ private void deleteDirWithFile(Path dir) throws IOException { * and, if the operation took place, the list of created dirs. * Reports progress on invocation. * @param dirEntry entry + * @throws PathIOException if after multiple attempts, the dest dir couldn't be created. * @throws IOException failure. */ private void createOneDirectory(final DirEntry dirEntry) throws IOException { @@ -270,9 +272,17 @@ private void createOneDirectory(final DirEntry dirEntry) throws IOException { * Try to efficiently and robustly create a directory in a method which is * expected to be executed in parallel with operations creating * peer directories. + * A return value of {@link DirMapState#dirWasCreated} or + * {@link DirMapState#dirCreatedOnSecondAttempt} indicates + * this thread did the creation. + * Other outcomes imply it already existed; if the directory + * cannot be created/found then a {@link PathIOException} is thrown. + * The outcome should be added to the {@link #dirMap} to avoid further creation attempts. * @param dirEntry dir to create - * @return Outcome - * @throws IOException IO Failure. + * @return Outcome of the operation, such as whether the entry was created, found in store. + * It will always be a success outcome of some form. + * @throws PathIOException if after multiple attempts, the dest dir couldn't be created. + * @throws IOException Other IO failure */ private DirMapState maybeCreateOneDirectory(DirEntry dirEntry) throws IOException { final EntryStatus status = dirEntry.getStatus(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer.md b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer.md index 3765efcd56d85..da199a48d14c0 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer.md +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer.md @@ -261,7 +261,7 @@ this queue size should not be a limit on manifest load performance. It can help limit the amount of memory consumed during manifest load during job commit. -The maximumum number of loaded manifests will be +The maximum number of loaded manifests will be: ``` mapreduce.manifest.committer.writer.queue.capacity + mapreduce.manifest.committer.io.threads From b289707c8174463a4715c8de9d9fa688913f7a02 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 1 Jun 2023 19:01:56 +0100 Subject: [PATCH 16/18] MAPREDUCE-7435. validation reporting missing files that is: success file contains entries which aren't present in the FS Fixes * find bit in earlier test where file was being deleted, and restore it (and re-order it too!) * LoadManifestsStage doesn't optionally return manifests for testing; tests modified to match. * EntryFileIO will report timeout after 10 minutes if queue blocks somehow. * LoadManifestsStage handles this timeout and will raise it as a failure, but only secondary to any exception raised by the writer thread * SUCCESS file can be configured with #of files to list, allows for tests to assert on many thousands of files, although in production it is still fixed to a small number for performance reasons. Change-Id: I642c1178928de427bf6e09f0fe0d345876311fb5 --- .../committer/manifest/impl/EntryFileIO.java | 28 ++++- .../manifest/stages/CommitJobStage.java | 24 ++-- .../manifest/stages/LoadManifestsStage.java | 118 +++++++++--------- .../manifest/stages/RenameFilesStage.java | 11 +- .../manifest/stages/StageConfig.java | 22 ++++ .../AbstractManifestCommitterTest.java | 1 + .../ManifestCommitterTestSupport.java | 23 ++-- .../TestJobThroughManifestCommitter.java | 61 +++++---- .../manifest/TestLoadManifestsStage.java | 7 +- .../manifest/TestRenameStageFailure.java | 7 +- 10 files changed, 183 insertions(+), 119 deletions(-) diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java index 93f73822b8c51..7f3d6b2ce68b0 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java @@ -63,7 +63,18 @@ public class EntryFileIO { private static final Logger LOG = LoggerFactory.getLogger( EntryFileIO.class); - public static final int WRITER_SHUTDOWN_TIMEOUT = 60; + /** + * How long should the writer shutdown take? + */ + public static final int WRITER_SHUTDOWN_TIMEOUT_SECONDS = 60; + + /** + * How long should trying to queue a write block before giving up + * with an error? + * This is a safety feature to ensure that if something has gone wrong + * in the queue code the job fails with an error rather than just hangs + */ + public static final int WRITER_QUEUE_PUT_TIMEOUT_MINUTES = 10; /** Configuration used to load filesystems. */ private final Configuration conf; @@ -327,15 +338,20 @@ public boolean enqueue(List entries) { } if (active.get()) { try { - queue.put(new QueueEntry(Actions.write, entries)); - LOG.debug("Queued {}", entries.size()); - return true; + LOG.debug("Queueing {} entries", entries.size()); + final boolean enqueued = queue.offer(new QueueEntry(Actions.write, entries), + WRITER_QUEUE_PUT_TIMEOUT_MINUTES, TimeUnit.MINUTES); + if (!enqueued) { + LOG.warn("Timeout submitting entries to {}", this); + } + return enqueued; } catch (InterruptedException e) { Thread.interrupted(); return false; } } else { - LOG.warn("EntryFile write queue inactive; discarding {} entries", entries.size()); + LOG.warn("EntryFile write queue inactive; discarding {} entries submitted to {}", + entries.size(), this); return false; } } @@ -424,7 +440,7 @@ public void close() throws IOException { } try { // wait for the op to finish. - int total = FutureIO.awaitFuture(future, WRITER_SHUTDOWN_TIMEOUT, TimeUnit.SECONDS); + int total = FutureIO.awaitFuture(future, WRITER_SHUTDOWN_TIMEOUT_SECONDS, TimeUnit.SECONDS); LOG.debug("Processed {} files", total); executor.shutdown(); } catch (TimeoutException e) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java index 35f0df2d08662..e657162918b55 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java @@ -26,7 +26,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.commons.lang3.tuple.Pair; +import org.apache.commons.lang3.tuple.Triple; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; @@ -35,6 +35,7 @@ import static java.util.Objects.requireNonNull; import static org.apache.commons.lang3.StringUtils.isNotBlank; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SUCCESS_MARKER_FILE_LIMIT; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_BYTES_COMMITTED_COUNT; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_FILES_COMMITTED_COUNT; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_COMMIT; @@ -84,23 +85,22 @@ protected CommitJobStage.Result executeStage( LoadManifestsStage.Result result = new LoadManifestsStage(stageConfig).apply( new LoadManifestsStage.Arguments( File.createTempFile("manifest", ".list"), - false, /* do not cache manifests */ + /* do not cache manifests */ stageConfig.getWriterQueueCapacity())); - LoadManifestsStage.SummaryInfo summary = result.getSummary(); + LoadManifestsStage.SummaryInfo loadedManifestSummary = result.getSummary(); loadedManifestData = result.getLoadedManifestData(); - LOG.debug("{}: Job Summary {}", getName(), summary); + LOG.debug("{}: Job Summary {}", getName(), loadedManifestSummary); LOG.info("{}: Committing job with file count: {}; total size {} bytes", getName(), - summary.getFileCount(), - String.format("%,d", summary.getTotalFileSize())); + loadedManifestSummary.getFileCount(), + String.format("%,d", loadedManifestSummary.getTotalFileSize())); addHeapInformation(heapInfo, OP_STAGE_JOB_LOAD_MANIFESTS); - // add in the manifest statistics to our local IOStatistics for // reporting. IOStatisticsStore iostats = getIOStatistics(); - iostats.aggregate(summary.getIOStatistics()); + iostats.aggregate(loadedManifestSummary.getIOStatistics()); // prepare destination directories. final CreateOutputDirectoriesStage.Result dirStageResults = @@ -113,7 +113,9 @@ protected CommitJobStage.Result executeStage( // and hence all aggregate stats from the tasks. ManifestSuccessData successData; successData = new RenameFilesStage(stageConfig).apply( - Pair.of(loadedManifestData, dirStageResults.getCreatedDirectories())); + Triple.of(loadedManifestData, + dirStageResults.getCreatedDirectories(), + stageConfig.getSuccessMarkerFileLimit())); if (LOG.isDebugEnabled()) { LOG.debug("{}: _SUCCESS file summary {}", getName(), successData.toJson()); } @@ -124,10 +126,10 @@ protected CommitJobStage.Result executeStage( // aggregating tasks. iostats.setCounter( COMMITTER_FILES_COMMITTED_COUNT, - summary.getFileCount()); + loadedManifestSummary.getFileCount()); iostats.setCounter( COMMITTER_BYTES_COMMITTED_COUNT, - summary.getTotalFileSize()); + loadedManifestSummary.getTotalFileSize()); successData.snapshotIOStatistics(iostats); successData.getIOStatistics().aggregate(heapInfo); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java index 1874c82232758..f1c4216dc6854 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java @@ -76,11 +76,6 @@ public class LoadManifestsStage extends */ private final SummaryInfo summaryInfo = new SummaryInfo(); - /** - * List of loaded manifests. - */ - private final List manifests = new ArrayList<>(); - /** * Map of directories from manifests, coalesced to reduce duplication. */ @@ -91,12 +86,6 @@ public class LoadManifestsStage extends */ private EntryFileIO.EntryWriter entryWriter; - /** - * Should the manifests be cached and returned? - * only for testing. - */ - private boolean cacheManifests; - public LoadManifestsStage(final StageConfig stageConfig) { super(false, stageConfig, OP_STAGE_JOB_LOAD_MANIFESTS, true); } @@ -117,7 +106,6 @@ protected LoadManifestsStage.Result executeStage( LOG.info("{}: Executing Manifest Job Commit with manifests in {}", getName(), manifestDir); - cacheManifests = arguments.cacheManifests; final Path entrySequenceData = arguments.getEntrySequenceData(); @@ -125,12 +113,7 @@ protected LoadManifestsStage.Result executeStage( entryWriter = entryFileIO.launchEntryWriter( entryFileIO.createWriter(entrySequenceData), arguments.queueCapacity); - // manifest list is only built up when caching is enabled. - // as this is memory hungry, it is warned about - List manifestList; - if (arguments.cacheManifests) { - LOG.info("Loaded manifests are cached; this is memory hungry"); - } + try { // sync fs before the list @@ -142,7 +125,7 @@ protected LoadManifestsStage.Result executeStage( haltableRemoteIterator(listManifests(), () -> entryWriter.isActive()); - manifestList = loadAllManifests(manifestFiles); + processAllManifests(manifestFiles); maybeAddIOStatistics(getIOStatistics(), manifestFiles); LOG.info("{}: Summary of {} manifests loaded in {}: {}", @@ -158,24 +141,34 @@ protected LoadManifestsStage.Result executeStage( 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, manifestList); + return new LoadManifestsStage.Result(summaryInfo, loadedManifestData); } /** - * Load all the manifests. + * Load and process all the manifests. * @param manifestFiles list of manifest files. - * @return the loaded manifests. - * @throws IOException IO Failure. + * @throws IOException failure to load/parse/queue */ - private List loadAllManifests( + private void processAllManifests( final RemoteIterator manifestFiles) throws IOException { trackDurationOfInvocation(getIOStatistics(), OP_LOAD_ALL_MANIFESTS, () -> @@ -183,13 +176,12 @@ private List loadAllManifests( .executeWith(getIOProcessors()) .stopOnFailure() .run(this::processOneManifest)); - return manifests; } /** * Method invoked to process one manifest. * @param status file to process. - * @throws IOException failure to load/parse + * @throws IOException failure to load/parse/queue */ private void processOneManifest(FileStatus status) throws IOException { @@ -200,9 +192,9 @@ private void processOneManifest(FileStatus status) // update the directories final int created = coalesceDirectories(manifest); - final String taskID = manifest.getTaskID(); - LOG.debug("{}: task {} added {} directories", - getName(), taskID, created); + final String attemptID = manifest.getTaskAttemptID(); + LOG.debug("{}: task attempt {} added {} directories", + getName(), attemptID, created); // add to the summary. summaryInfo.add(manifest); @@ -213,20 +205,12 @@ private void processOneManifest(FileStatus status) manifest.setIOStatistics(null); manifest.getExtraData().clear(); - // if manifests are cached add to the list - if (cacheManifests) { - // update the manifest list in a synchronized block. - synchronized (manifests) { - manifests.add(manifest); - } - } - // queue those files. final boolean enqueued = entryWriter.enqueue(manifest.getFilesToCommit()); if (!enqueued) { LOG.warn("{}: Failed to write manifest for task {}", - getName(), - taskID); + getName(), attemptID); + throw new EntryWriteException(attemptID); } } @@ -301,21 +285,20 @@ public static final class Arguments { */ private final File entrySequenceFile; - /** - * build a list of manifests and return them? - */ - private final boolean cacheManifests; - /** * Capacity for queue between manifest loader and the writers. */ private final int queueCapacity; - public Arguments(final File entrySequenceFile, - final boolean cacheManifests, + /** + * 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.cacheManifests = cacheManifests; this.queueCapacity = queueCapacity; } @@ -331,21 +314,20 @@ private Path getEntrySequenceData() { public static final class Result { private final SummaryInfo summary; - /** - * manifest list, non-null only if cacheManifests is true. - */ - private final List manifests; - /** * Output of this stage to pass on to the subsequence stages. */ private final LoadedManifestData loadedManifestData; - public Result(SummaryInfo summary, - final LoadedManifestData loadedManifestData, - final List 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.manifests = manifests; this.loadedManifestData = loadedManifestData; } @@ -353,15 +335,21 @@ public SummaryInfo getSummary() { return summary; } - public List 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. * Implementation note: atomic counters are used here to keep spotbugs quiet, @@ -379,6 +367,11 @@ public static final class SummaryInfo implements IOStatisticsSource { */ private final List taskIDs = new ArrayList<>(); + /** + * Task IDs. + */ + private final List taskAttemptIDs = new ArrayList<>(); + /** * How many manifests were loaded. */ @@ -431,6 +424,10 @@ public List getTaskIDs() { return taskIDs; } + public List getTaskAttemptIDs() { + return taskAttemptIDs; + } + /** * Add all statistics; synchronized. * @param manifest manifest to add. @@ -442,6 +439,7 @@ public synchronized void add(TaskManifest manifest) { directoryCount.addAndGet(manifest.getDestDirectories().size()); totalFileSize.addAndGet(manifest.getTotalFileSize()); taskIDs.add(manifest.getTaskID()); + taskAttemptIDs.add(manifest.getTaskAttemptID()); } /** diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/RenameFilesStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/RenameFilesStage.java index a8cd74e938776..0686f55d91852 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/RenameFilesStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/RenameFilesStage.java @@ -27,7 +27,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.commons.lang3.tuple.Pair; +import org.apache.commons.lang3.tuple.Triple; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry; @@ -36,7 +36,6 @@ import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.LoadedManifestData; import org.apache.hadoop.util.functional.TaskPool; -import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SUCCESS_MARKER_FILE_LIMIT; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_COMMIT; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_RENAME_FILES; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.createManifestOutcome; @@ -59,7 +58,7 @@ */ public class RenameFilesStage extends AbstractJobOrTaskStage< - Pair>, + Triple, Integer>, ManifestSuccessData> { private static final Logger LOG = LoggerFactory.getLogger( @@ -106,12 +105,12 @@ public synchronized long getTotalFileSize() { */ @Override protected ManifestSuccessData executeStage( - Pair> args) + Triple, Integer> args) throws IOException { final LoadedManifestData manifestData = args.getLeft(); - createdDirectories = args.getRight(); + createdDirectories = args.getMiddle(); final EntryFileIO entryFileIO = new EntryFileIO(getStageConfig().getConf()); @@ -140,7 +139,7 @@ protected ManifestSuccessData executeStage( // enough for simple testing success.setFilenamePaths( committed - .subList(0, Math.min(committed.size(), SUCCESS_MARKER_FILE_LIMIT)) + .subList(0, Math.min(committed.size(), args.getRight())) .stream().map(FileEntry::getDestPath) .collect(Collectors.toList())); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/StageConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/StageConfig.java index 9ad121c2761c2..b716d2f4b7f0c 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/StageConfig.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/StageConfig.java @@ -31,6 +31,7 @@ import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.DEFAULT_WRITER_QUEUE_CAPACITY; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SUCCESS_MARKER; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SUCCESS_MARKER_FILE_LIMIT; /** * Stage Config. @@ -166,6 +167,11 @@ public class StageConfig { */ private int writerQueueCapacity = DEFAULT_WRITER_QUEUE_CAPACITY; + /** + * Number of marker files to include in success file. + */ + private int successMarkerFileLimit = SUCCESS_MARKER_FILE_LIMIT; + public StageConfig() { } @@ -582,6 +588,22 @@ public boolean getDeleteTargetPaths() { return deleteTargetPaths; } + /** + * Number of marker files to include in success file. + * @param value new value + * @return the builder + */ + public StageConfig withSuccessMarkerFileLimit(final int value) { + checkOpen(); + + successMarkerFileLimit = value; + return this; + } + + public int getSuccessMarkerFileLimit() { + return successMarkerFileLimit; + } + /** * Enter the stage; calls back to * {@link #enterStageEventHandler} if non-null. diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/AbstractManifestCommitterTest.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/AbstractManifestCommitterTest.java index 514f2fe8ec7ce..5b64d544bc551 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/AbstractManifestCommitterTest.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/AbstractManifestCommitterTest.java @@ -798,6 +798,7 @@ protected StageConfig createStageConfig( .withName(String.format(NAME_FORMAT_JOB_ATTEMPT, jobId)) .withOperations(getStoreOperations()) .withProgressable(getProgressCounter()) + .withSuccessMarkerFileLimit(100_000) .withWriterQueueCapacity(DEFAULT_WRITER_QUEUE_CAPACITY); // if there's a task attempt ID set, set up its details diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterTestSupport.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterTestSupport.java index a398bf1da58eb..cff215f300c41 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterTestSupport.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterTestSupport.java @@ -24,9 +24,11 @@ import java.io.PrintStream; import java.time.LocalDateTime; import java.time.format.DateTimeFormatter; +import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; import org.assertj.core.api.Assertions; @@ -53,6 +55,7 @@ import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.LoadedManifestData; import org.apache.hadoop.util.functional.RemoteIterators; +import static java.util.Comparator.naturalOrder; import static org.apache.commons.lang3.StringUtils.isNotEmpty; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.MANIFEST_COMMITTER_CLASSNAME; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SUCCESS_MARKER; @@ -205,29 +208,35 @@ public static Map validateGeneratedFiles( Path destDir, ManifestSuccessData successData, boolean exclusive) throws IOException { - Map map = new HashMap<>(); + Map fileListing = new HashMap<>(); RemoteIterators.foreach(fs.listFiles(destDir, true), e -> { if (!e.getPath().getName().startsWith("_")) { - map.put(e.getPath(), e); + fileListing.put(e.getPath(), e); } }); + final List actual = fileListing.keySet().stream() + .sorted(Comparator.comparing(Path::getName)) + .collect(Collectors.toList()); + // map has all files other than temp ones and the success marker // what do we expect final List expected = filesInManifest(successData); + expected.sort(Comparator.comparing(Path::getName)); // all of those must be found - Assertions.assertThat(map.keySet()) - .describedAs("Files in FS compared to manifest") + Assertions.assertThat(actual) + .describedAs("Files in FS expected to contain all listed in manifest") .containsAll(expected); // and if exclusive, that too if (exclusive) { - Assertions.assertThat(map.keySet()) - .describedAs("Files in FS compared to manifest") + Assertions.assertThat(actual) + .describedAs("Files in FS expected to be exclusively of the job") + .hasSize(expected.size()) .containsExactlyInAnyOrderElementsOf(expected); } - return map; + return fileListing; } /** diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java index 08011a20c5aa2..20870684815fd 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java @@ -22,7 +22,6 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.util.List; -import java.util.Map; import java.util.stream.Collectors; import org.assertj.core.api.Assertions; @@ -145,7 +144,10 @@ public class TestJobThroughManifestCommitter */ private StageConfig ta11Config; - private LoadedManifestData + /** + * Loaded manifest data, set in job commit and used in validation. + */ + private static LoadedManifestData loadedManifestData; @Override @@ -451,23 +453,22 @@ public void test_0400_loadManifests() throws Throwable { describe("Load all manifests; committed must be TA01 and TA10"); File entryFile = File.createTempFile("entry", ".seq"); LoadManifestsStage.Arguments args = new LoadManifestsStage.Arguments( - entryFile, true, DEFAULT_WRITER_QUEUE_CAPACITY); + entryFile, DEFAULT_WRITER_QUEUE_CAPACITY); LoadManifestsStage.Result result = new LoadManifestsStage(getJobStageConfig()).apply(args); loadedManifestData = result.getLoadedManifestData(); + Assertions.assertThat(loadedManifestData) + .describedAs("manifest data from %s", result) + .isNotNull(); - String summary = result.getSummary().toString(); + final LoadManifestsStage.SummaryInfo stageSummary = result.getSummary(); + String summary = stageSummary.toString(); LOG.info("Manifest summary {}", summary); - List manifests = result.getManifests(); - Assertions.assertThat(manifests) - .describedAs("Loaded manifests in %s", summary) - .hasSize(2); - Map manifestMap = toMap(manifests); - verifyManifestTaskAttemptID( - manifestMap.get(taskAttempt01), taskAttempt01); - verifyManifestTaskAttemptID( - manifestMap.get(taskAttempt10), taskAttempt10); + Assertions.assertThat(stageSummary.getTaskAttemptIDs()) + .describedAs("Task attempts in %s", summary) + .hasSize(2) + .contains(taskAttempt01, taskAttempt10); } @Test @@ -490,13 +491,11 @@ public void test_0420_validateJob() throws Throwable { .describedAs("Loaded Manifest Data from earlier stage") .isNotNull(); - // load in the success data. ManifestSuccessData successData = loadAndPrintSuccessData( getFileSystem(), getJobStageConfig().getJobSuccessMarkerPath()); - // Now verify their files exist, returning the list of renamed files. final List validatedEntries = new ValidateRenamedFilesStage(getJobStageConfig()) .apply(loadedManifestData.getEntrySequenceData()); @@ -513,11 +512,7 @@ public void test_0420_validateJob() throws Throwable { Assertions.assertThat(committedFiles) .containsAll(successData.getFilenames()); - // delete an entry, repeat - getFileSystem().delete(validatedEntries.get(0).getDestPath(), false); - intercept(OutputValidationException.class, () -> - new ValidateRenamedFilesStage(getJobStageConfig()) - .apply(loadedManifestData.getEntrySequenceData())); + } @Test @@ -561,7 +556,7 @@ public void test_0430_validateStatistics() throws Throwable { } @Test - public void test_440_validateSuccessFiles() throws Throwable { + public void test_0440_validateSuccessFiles() throws Throwable { // load in the success data. final FileSystem fs = getFileSystem(); @@ -573,6 +568,30 @@ public void test_440_validateSuccessFiles() throws Throwable { successData, false); } + /** + * Verify that the validation stage will correctly report a failure + * if one of the files has as different name. + */ + + @Test + public void test_0450_validationDetectsFailures() throws Throwable { + // delete an entry, repeat + final List validatedEntries = new ValidateRenamedFilesStage(getJobStageConfig()) + .apply(loadedManifestData.getEntrySequenceData()); + final Path path = validatedEntries.get(0).getDestPath(); + final Path p2 = new Path(path.getParent(), path.getName() + "-renamed"); + final FileSystem fs = getFileSystem(); + fs.rename(path, p2); + try { + intercept(OutputValidationException.class, () -> + new ValidateRenamedFilesStage(getJobStageConfig()) + .apply(loadedManifestData.getEntrySequenceData())); + } finally { + // if this doesn't happen, later stages will fail. + fs.rename(p2, path); + } + } + @Test public void test_0900_cleanupJob() throws Throwable { describe("Cleanup job"); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestLoadManifestsStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestLoadManifestsStage.java index 9b869d4c2e4cc..4dd7fe2dbcea5 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestLoadManifestsStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestLoadManifestsStage.java @@ -133,7 +133,7 @@ public void testSaveThenLoadManyManifests() throws Throwable { stageConfig); entryFile = File.createTempFile("entry", ".seq"); LoadManifestsStage.Arguments args = new LoadManifestsStage.Arguments( - entryFile, false, DEFAULT_WRITER_QUEUE_CAPACITY); + entryFile, DEFAULT_WRITER_QUEUE_CAPACITY); LoadManifestsStage.Result loadManifestsResult = stage.apply(args); LoadManifestsStage.SummaryInfo summary = loadManifestsResult.getSummary(); @@ -143,7 +143,6 @@ public void testSaveThenLoadManyManifests() throws Throwable { LOG.info("Heap size = {}", heapSize()); heapinfo(heapInfo, "load.manifests"); - Assertions.assertThat(summary.getManifestCount()) .describedAs("Manifest count of %s", summary) .isEqualTo(taskAttemptCount); @@ -193,9 +192,7 @@ public void testSaveThenLoadManyManifests() throws Throwable { final FileSystem summaryFS = path.getFileSystem(conf); success.save(summaryFS, path, true); LOG.info("Saved summary to {}", path); - ManifestPrinter showManifest = new ManifestPrinter(); - ManifestSuccessData manifestSuccessData = - showManifest.loadAndPrintManifest(summaryFS, path); + new ManifestPrinter().loadAndPrintManifest(summaryFS, path); } /** diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestRenameStageFailure.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestRenameStageFailure.java index 9a482489d5085..749ae406da790 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestRenameStageFailure.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestRenameStageFailure.java @@ -29,7 +29,7 @@ import org.junit.Assume; import org.junit.Test; -import org.apache.commons.lang3.tuple.Pair; +import org.apache.commons.lang3.tuple.Triple; import org.apache.hadoop.fs.CommonPathCapabilities; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -50,6 +50,7 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyFileContents; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter; import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SUCCESS_MARKER_FILE_LIMIT; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_COMMIT_FILE_RENAME; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterTestSupport.saveManifest; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.getEtag; @@ -243,7 +244,7 @@ public void testDeleteTargetPaths() throws Throwable { // delete target paths and it works try { new RenameFilesStage(stageConfig.withDeleteTargetPaths(true)) - .apply(Pair.of(manifestData, Collections.emptySet())); + .apply(Triple.of(manifestData, Collections.emptySet(), SUCCESS_MARKER_FILE_LIMIT)); } finally { manifestData.getEntrySequenceFile().delete(); } @@ -365,7 +366,7 @@ private E expectRenameFailure( E ex; try { ex = intercept(exceptionClass, errorText, () -> - stage.apply(Pair.of(manifestData, Collections.emptySet()))); + stage.apply(Triple.of(manifestData, Collections.emptySet(), SUCCESS_MARKER_FILE_LIMIT))); } finally { manifestData.getEntrySequenceFile().delete(); } From 355fa358820d44d29f5852a03b767a6334ebbf03 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 1 Jun 2023 20:07:05 +0100 Subject: [PATCH 17/18] MAPREDUCE-7435. Mehakmeet review Change-Id: Ica813c6068eca18d83bf2f5f94fac4a1e1996c36 --- .../fs/statistics/IOStatisticsSetters.java | 2 +- .../statistics/TestIOStatisticsSetters.java | 29 +++++++++++++++---- .../manifest/impl/InternalConstants.java | 1 - .../TestJobThroughManifestCommitter.java | 2 ++ .../manifest/TestRenameStageFailure.java | 3 ++ .../manifest/impl/TestEntryFileIO.java | 1 + 6 files changed, 30 insertions(+), 8 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSetters.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSetters.java index 402ea27235408..1d1cf9687e7ab 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSetters.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSetters.java @@ -29,7 +29,7 @@ * {@link IOStatisticsSnapshot} to also support it. * These are the simple setters, they don't provide for increments, * decrements, calculation of min/max/mean etc. - * @since The interface and IOStatisticsSnapshot support was added after Hadoop 3.3.5 + * @since The interface and IOStatisticsSnapshot support was added after Hadoop 3.3.5 */ @InterfaceAudience.Public @InterfaceStability.Evolving diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsSetters.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsSetters.java index e8fc033f10ec7..7dfb540500457 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsSetters.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsSetters.java @@ -21,6 +21,7 @@ import java.util.Arrays; import java.util.Collection; +import org.assertj.core.api.Assertions; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -61,12 +62,14 @@ public class TestIOStatisticsSetters extends AbstractHadoopTestBase { private final IOStatisticsSetters ioStatistics; - @Parameterized.Parameters + private final boolean createsNewEntries; + + @Parameterized.Parameters(name="{0}") public static Collection params() { return Arrays.asList(new Object[][]{ - {new IOStatisticsSnapshot()}, - {createTestStore()}, - {new ForwardingIOStatisticsStore(createTestStore())}, + {"IOStatisticsSnapshot", new IOStatisticsSnapshot(), true}, + {"IOStatisticsStore", createTestStore(), false}, + {"ForwardingIOStatisticsStore", new ForwardingIOStatisticsStore(createTestStore()), false}, }); } @@ -84,8 +87,13 @@ private static IOStatisticsStore createTestStore() { .build(); } - public TestIOStatisticsSetters(IOStatisticsSetters ioStatisticsSetters) { + public TestIOStatisticsSetters( + String source, + IOStatisticsSetters ioStatisticsSetters, + boolean createsNewEntries) { this.ioStatistics = ioStatisticsSetters; + + this.createsNewEntries = createsNewEntries; } @Test @@ -101,7 +109,16 @@ public void testCounter() throws Throwable { .isEqualTo(2); // unknown value - ioStatistics.setCounter("c2", 3); + final String unknown = "unknown"; + ioStatistics.setCounter(unknown, 3); + if (createsNewEntries) { + assertThatStatisticCounter(ioStatistics, unknown) + .isEqualTo(3); + } else { + Assertions.assertThat(ioStatistics.counters()) + .describedAs("Counter map in {}", ioStatistics) + .doesNotContainKey(unknown); + } } @Test diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/InternalConstants.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/InternalConstants.java index 7155432c04287..15f9899f3551e 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/InternalConstants.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/InternalConstants.java @@ -127,5 +127,4 @@ private InternalConstants() { /** Schemas of filesystems we know to not work with this committer. */ public static final Set UNSUPPORTED_FS_SCHEMAS = ImmutableSet.of("s3a", "wasb"); - } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java index 20870684815fd..4bc2ce9bcf648 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java @@ -146,6 +146,8 @@ public class TestJobThroughManifestCommitter /** * Loaded manifest data, set in job commit and used in validation. + * This is static so it can be passed from where it is loaded + * {@link #test_0400_loadManifests()} to subsequent tests. */ private static LoadedManifestData loadedManifestData; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestRenameStageFailure.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestRenameStageFailure.java index 749ae406da790..5c80aee5b5f98 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestRenameStageFailure.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestRenameStageFailure.java @@ -86,6 +86,9 @@ public class TestRenameStageFailure extends AbstractManifestCommitterTest { /** resilient commit expected? */ private boolean resilientCommit; + /** + * Entry file IO. + */ private EntryFileIO entryFileIO; protected boolean isResilientCommit() { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/TestEntryFileIO.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/TestEntryFileIO.java index dff7acee7be32..93f5050287551 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/TestEntryFileIO.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/TestEntryFileIO.java @@ -193,6 +193,7 @@ public void testCreateEmptyFile() throws Throwable { // now use the iterator to access it. List files = new ArrayList<>(); Assertions.assertThat(foreach(iterateOverEntryFile(), files::add)) + .describedAs("Count of iterations over entries in an entry file with no entries") .isEqualTo(0); } From 070c7881df7694847844da286d065d558eb3965d Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 6 Jun 2023 17:30:04 +0100 Subject: [PATCH 18/18] MAPREDUCE-7435. checkstyle: remove unused imports. The other new ones are related to test methods whose numbering breaks the style checker's requirements * test_0440_validateSuccessFiles * test_0450_validationDetectsFailures Change-Id: I36267e4d9912873e457126341385f866acd6d148 --- .../lib/output/committer/manifest/stages/CommitJobStage.java | 1 - .../output/committer/manifest/ManifestCommitterTestSupport.java | 2 -- 2 files changed, 3 deletions(-) diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java index e657162918b55..60fc6492ee621 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java @@ -35,7 +35,6 @@ import static java.util.Objects.requireNonNull; import static org.apache.commons.lang3.StringUtils.isNotBlank; -import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SUCCESS_MARKER_FILE_LIMIT; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_BYTES_COMMITTED_COUNT; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_FILES_COMMITTED_COUNT; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_COMMIT; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterTestSupport.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterTestSupport.java index cff215f300c41..3b52fe9875641 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterTestSupport.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterTestSupport.java @@ -28,7 +28,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.stream.Collectors; import org.assertj.core.api.Assertions; @@ -55,7 +54,6 @@ import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.LoadedManifestData; import org.apache.hadoop.util.functional.RemoteIterators; -import static java.util.Comparator.naturalOrder; import static org.apache.commons.lang3.StringUtils.isNotEmpty; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.MANIFEST_COMMITTER_CLASSNAME; import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SUCCESS_MARKER;