steveloughran commented on a change in pull request #2971: URL: https://github.com/apache/hadoop/pull/2971#discussion_r827957400
########## File path: hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CreateOutputDirectoriesStage.java ########## @@ -0,0 +1,421 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages; + +import java.io.IOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FileStatus; +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; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_DELETE; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.measureDurationOfInvocation; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_CREATE_DIRECTORIES; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_DELETE_FILE_UNDER_DESTINATION; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_MKDIRS_RETURNED_FALSE; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_PREPARE_DIR_ANCESTORS; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_CREATE_TARGET_DIRS; +import static org.apache.hadoop.util.OperationDuration.humanTime; + +/** + * Prepare the destination directory tree, as efficiently as possible. + * possible -and doing those IO operations in the thread pool. + * + * The classic FileOutputCommitter does a recursive treewalk and + * deletes any files found at paths where directories are to be created. + * + * Each task manifest's directories are combined with those of the other tasks + * to build a set of all directories which are needed, without duplicates. + * + * This stage requires the aggregate set of manifests to contain + * all directories to create, including level, + * and expects them to have been probed for existence/state. + * + * For each level, all dirs are processed in parallel to + * be created or, if files, deleted. + * + * The stage returns the list of directories created, and for testing, + * the map of paths to outcomes. + * + * Directory creation can be surprisingly slow against object stores, + * do use benchmarks from real test runs when tuning this algorithm. + */ +public class CreateOutputDirectoriesStage extends + AbstractJobCommitStage<List<TaskManifest>, 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. + */ + private final Map<Path, DirMapState> dirMap = new ConcurrentHashMap<>(); + + /** + * A list of created paths for the results. + */ + private final List<Path> createdDirectories = new ArrayList<>(); + + public CreateOutputDirectoriesStage(final StageConfig stageConfig) { + super(false, stageConfig, OP_STAGE_JOB_CREATE_TARGET_DIRS, true); + // add the dest dir to the dir map as we expect the job setup to create it. + dirMap.put(getDestinationDir(), DirMapState.dirWasCreated); + } + + @Override + protected Result executeStage( + final List<TaskManifest> taskManifests) + throws IOException { + + final List<Path> directories = createAllDirectories(taskManifests); + 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 + * @return the list of paths which have been created. + */ + private List<Path> createAllDirectories(final List<TaskManifest> taskManifests) + throws IOException { + + // all directories which need to exist across all + // tasks. + // leaf directories + final Map<Path, DirEntry> leaves = new HashMap<>(); + // parent directories. these do not need to be + // explicitly created. + final Map<Path, DirEntry> parents = new HashMap<>(); + // the files which must be deleted as a directory + // will be created at that path. + final Set<Path> 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<DirEntry> 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) { Review comment: that list of dirs is built up in task commit, where each task checks the status of the equivalent dir in the dest path. so we know which target dirs have files, as well as which don't exist -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
