diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/AuditConstants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/AuditConstants.java index d9629e388b..0929c2be03 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/AuditConstants.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/AuditConstants.java @@ -90,6 +90,11 @@ private AuditConstants() { */ public static final String PARAM_PROCESS = "ps"; + /** + * Task Attempt ID query header: {@value}. + */ + public static final String PARAM_TASK_ATTEMPT_ID = "ta"; + /** * Thread 0: the thread which created a span {@value}. */ diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/CommonAuditContext.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/CommonAuditContext.java index e188e168e5..2dcd4f8b3f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/CommonAuditContext.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/CommonAuditContext.java @@ -124,11 +124,15 @@ private CommonAuditContext() { /** * Put a context entry. * @param key key - * @param value new value + * @param value new value., If null, triggers removal. * @return old value or null */ public Supplier put(String key, String value) { - return evaluatedEntries.put(key, () -> value); + if (value != null) { + return evaluatedEntries.put(key, () -> value); + } else { + return evaluatedEntries.remove(key); + } } /** diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java index c458269c35..c04c1bb47f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java @@ -53,6 +53,9 @@ public final class StoreStatisticNames { /** {@value}. */ public static final String OP_CREATE = "op_create"; + /** {@value}. */ + public static final String OP_CREATE_FILE = "op_createfile"; + /** {@value}. */ public static final String OP_CREATE_NON_RECURSIVE = "op_create_non_recursive"; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java index 0bba79fd77..52c6c45052 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java @@ -297,11 +297,12 @@ public void save(FileSystem fs, Path path, T instance, } /** - * Write the JSON as bytes, then close the file. + * Write the JSON as bytes, then close the stream. + * @param instance instance to write * @param dataOutputStream an output stream that will always be closed * @throws IOException on any failure */ - private void writeJsonAsBytes(T instance, + public void writeJsonAsBytes(T instance, OutputStream dataOutputStream) throws IOException { try { dataOutputStream.write(toBytes(instance)); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CloseableTaskPoolSubmitter.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CloseableTaskPoolSubmitter.java index 26b687a3c5..695da7e932 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CloseableTaskPoolSubmitter.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CloseableTaskPoolSubmitter.java @@ -34,7 +34,7 @@ */ @InterfaceAudience.Public @InterfaceStability.Unstable -public final class CloseableTaskPoolSubmitter implements TaskPool.Submitter, +public class CloseableTaskPoolSubmitter implements TaskPool.Submitter, Closeable { /** Executors. */ diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdataoutputstreambuilder.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdataoutputstreambuilder.md index 64dda2df8c..59a93c5887 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdataoutputstreambuilder.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdataoutputstreambuilder.md @@ -26,7 +26,7 @@ create a new file or open an existing file on `FileSystem` for write. ## Invariants The `FSDataOutputStreamBuilder` interface does not validate parameters -and modify the state of `FileSystem` until [`build()`](#Builder.build) is +and modify the state of `FileSystem` until `build()` is invoked. ## Implementation-agnostic parameters. @@ -110,7 +110,7 @@ of `FileSystem`. #### Implementation Notes The concrete `FileSystem` and/or `FSDataOutputStreamBuilder` implementation -MUST verify that implementation-agnostic parameters (i.e., "syncable") or +MUST verify that implementation-agnostic parameters (i.e., "syncable`) or implementation-specific parameters (i.e., "foofs:cache") are supported. `FileSystem` will satisfy optional parameters (via `opt(key, ...)`) on best effort. If the mandatory parameters (via `must(key, ...)`) can not be satisfied @@ -182,3 +182,58 @@ see `FileSystem#create(path, ...)` and `FileSystem#append()`. result = FSDataOutputStream The result is `FSDataOutputStream` to be used to write data to filesystem. + + +## S3A-specific options + +Here are the custom options which the S3A Connector supports. + +| Name | Type | Meaning | +|-----------------------------|-----------|----------------------------------------| +| `fs.s3a.create.performance` | `boolean` | create a file with maximum performance | +| `fs.s3a.create.header` | `string` | prefix for user supplied headers | + +### `fs.s3a.create.performance` + +Prioritize file creation performance over safety checks for filesystem consistency. + +This: +1. Skips the `LIST` call which makes sure a file is being created over a directory. + Risk: a file is created over a directory. +1. Ignores the overwrite flag. +1. Never issues a `DELETE` call to delete parent directory markers. + +It is possible to probe an S3A Filesystem instance for this capability through +the `hasPathCapability(path, "fs.s3a.create.performance")` check. + +Creating files with this option over existing directories is likely +to make S3A filesystem clients behave inconsistently. + +Operations optimized for directories (e.g. listing calls) are likely +to see the directory tree not the file; operations optimized for +files (`getFileStatus()`, `isFile()`) more likely to see the file. +The exact form of the inconsistencies, and which operations/parameters +trigger this are undefined and may change between even minor releases. + +Using this option is the equivalent of pressing and holding down the +"Electronic Stability Control" +button on a rear-wheel drive car for five seconds: the safety checks are off. +Things wil be faster if the driver knew what they were doing. +If they didn't, the fact they had held the button down will +be used as evidence at the inquest as proof that they made a +conscious decision to choose speed over safety and +that the outcome was their own fault. + +Accordingly: *Use if and only if you are confident that the conditions are met.* + +### `fs.s3a.create.header` User-supplied header support + +Options with the prefix `fs.s3a.create.header.` will be added to to the +S3 object metadata as "user defined metadata". +This metadata is visible to all applications. It can also be retrieved through the +FileSystem/FileContext `listXAttrs()` and `getXAttrs()` API calls with the prefix `header.` + +When an object is renamed, the metadata is propagated the copy created. + +It is possible to probe an S3A Filesystem instance for this capability through +the `hasPathCapability(path, "fs.s3a.create.header")` check. \ No newline at end of file diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java index eb56d957d9..90e3649c1b 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java @@ -29,6 +29,8 @@ import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.util.functional.RemoteIterators; + import org.junit.Assert; import org.junit.AssumptionViolatedException; import org.slf4j.Logger; @@ -1446,11 +1448,7 @@ public static TreeScanResults treeWalk(FileSystem fs, Path path) */ public static List toList( RemoteIterator iterator) throws IOException { - ArrayList list = new ArrayList<>(); - while (iterator.hasNext()) { - list.add(iterator.next()); - } - return list; + return RemoteIterators.toList(iterator); } /** @@ -1464,11 +1462,7 @@ public static List toList( */ public static List iteratorToList( RemoteIterator iterator) throws IOException { - List list = new ArrayList<>(); - while (iterator.hasNext()) { - list.add(iterator.next()); - } - return list; + return RemoteIterators.toList(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/files/AbstractManifestData.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/AbstractManifestData.java index 7020d5ca2d..bf67281785 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/AbstractManifestData.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/AbstractManifestData.java @@ -70,7 +70,6 @@ public static Path unmarshallPath(String path) { throw new RuntimeException( "Failed to parse \"" + path + "\" : " + e, e); - } } diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index 5583bb7ad0..6d9085b51a 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -405,6 +405,39 @@ + + org.apache.maven.plugins + maven-enforcer-plugin + + + banned-illegal-imports + process-sources + + enforce + + + + + false + Restrict mapreduce imports to committer code + + org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter + org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitterFactory + org.apache.hadoop.fs.s3a.commit.S3ACommitterFactory + org.apache.hadoop.fs.s3a.commit.impl.* + org.apache.hadoop.fs.s3a.commit.magic.* + org.apache.hadoop.fs.s3a.commit.staging.* + + + org.apache.hadoop.mapreduce.** + org.apache.hadoop.mapred.** + + + + + + + diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java index 2e4f1e8361..48d17aa402 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java @@ -1159,4 +1159,22 @@ private Constants() { * Require that all S3 access is made through Access Points. */ public static final String AWS_S3_ACCESSPOINT_REQUIRED = "fs.s3a.accesspoint.required"; + + /** + * Flag for create performance. + * This is *not* a configuration option; it is for use in the + * {code createFile()} builder. + * Value {@value}. + */ + public static final String FS_S3A_CREATE_PERFORMANCE = "fs.s3a.create.performance"; + + /** + * Prefix for adding a header to the object when created. + * The actual value must have a "." suffix and then the actual header. + * This is *not* a configuration option; it is only for use in the + * {code createFile()} builder. + * Value {@value}. + */ + public static final String FS_S3A_CREATE_HEADER = "fs.s3a.create.header"; + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java index 6e6871751d..8b1865c77c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java @@ -40,6 +40,7 @@ import com.amazonaws.services.s3.model.PutObjectResult; import com.amazonaws.services.s3.model.UploadPartRequest; +import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.Futures; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListenableFuture; @@ -68,6 +69,7 @@ import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.Statistic.*; import static org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext.EMPTY_BLOCK_OUTPUT_STREAM_STATISTICS; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; @@ -103,6 +105,11 @@ class S3ABlockOutputStream extends OutputStream implements /** IO Statistics. */ private final IOStatistics iostatistics; + /** + * The options this instance was created with. + */ + private final BlockOutputStreamBuilder builder; + /** Total bytes for uploads submitted so far. */ private long bytesSubmitted; @@ -167,6 +174,7 @@ class S3ABlockOutputStream extends OutputStream implements S3ABlockOutputStream(BlockOutputStreamBuilder builder) throws IOException { builder.validate(); + this.builder = builder; this.key = builder.key; this.blockFactory = builder.blockFactory; this.blockSize = (int) builder.blockSize; @@ -332,6 +340,7 @@ public synchronized void write(byte[] source, int offset, int len) * initializing the upload, or if a previous operation * has failed. */ + @Retries.RetryTranslated private synchronized void uploadCurrentBlock(boolean isLast) throws IOException { Preconditions.checkState(hasActiveBlock(), "No active block"); @@ -353,6 +362,7 @@ private synchronized void uploadCurrentBlock(boolean isLast) * can take time and potentially fail. * @throws IOException failure to initialize the upload */ + @Retries.RetryTranslated private void initMultipartUpload() throws IOException { if (multiPartUpload == null) { LOG.debug("Initiating Multipart upload"); @@ -546,9 +556,15 @@ private int putObject() throws IOException { int size = block.dataSize(); final S3ADataBlocks.BlockUploadData uploadData = block.startUpload(); final PutObjectRequest putObjectRequest = uploadData.hasFile() ? - writeOperationHelper.createPutObjectRequest(key, uploadData.getFile()) - : writeOperationHelper.createPutObjectRequest(key, - uploadData.getUploadStream(), size, null); + writeOperationHelper.createPutObjectRequest( + key, + uploadData.getFile(), + builder.putOptions) + : writeOperationHelper.createPutObjectRequest( + key, + uploadData.getUploadStream(), + size, + builder.putOptions); BlockUploadProgress callback = new BlockUploadProgress( block, progressListener, now()); @@ -559,7 +575,7 @@ private int putObject() throws IOException { try { // the putObject call automatically closes the input // stream afterwards. - return writeOperationHelper.putObject(putObjectRequest); + return writeOperationHelper.putObject(putObjectRequest, builder.putOptions); } finally { cleanupWithLogger(LOG, uploadData, block); } @@ -702,8 +718,21 @@ private class MultiPartUpload { */ private IOException blockUploadFailure; + /** + * Constructor. + * Initiates the MPU request against S3. + * @param key upload destination + * @throws IOException failure + */ + + @Retries.RetryTranslated MultiPartUpload(String key) throws IOException { - this.uploadId = writeOperationHelper.initiateMultiPartUpload(key); + this.uploadId = trackDuration(statistics, + OBJECT_MULTIPART_UPLOAD_INITIATED.getSymbol(), + () -> writeOperationHelper.initiateMultiPartUpload( + key, + builder.putOptions)); + this.partETagsFutures = new ArrayList<>(2); LOG.debug("Initiated multi-part upload for {} with " + "id '{}'", writeOperationHelper, uploadId); @@ -887,7 +916,8 @@ private void complete(List partETags) uploadId, partETags, bytesSubmitted, - errorCount); + errorCount, + builder.putOptions); }); } finally { statistics.exceptionInMultipartComplete(errorCount.get()); @@ -1057,6 +1087,11 @@ public static final class BlockOutputStreamBuilder { /** is Client side Encryption enabled? */ private boolean isCSEEnabled; + /** + * Put object options. + */ + private PutObjectOptions putOptions; + private BlockOutputStreamBuilder() { } @@ -1070,6 +1105,7 @@ public void validate() { requireNonNull(statistics, "null statistics"); requireNonNull(writeOperations, "null writeOperationHelper"); requireNonNull(putTracker, "null putTracker"); + requireNonNull(putOptions, "null putOptions"); Preconditions.checkArgument(blockSize >= Constants.MULTIPART_MIN_SIZE, "Block size is too small: %s", blockSize); } @@ -1182,5 +1218,16 @@ public BlockOutputStreamBuilder withCSEEnabled(boolean value) { isCSEEnabled = value; return this; } + + /** + * Set builder value. + * @param value new value + * @return the builder + */ + public BlockOutputStreamBuilder withPutOptions( + final PutObjectOptions value) { + putOptions = value; + return this; + } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 831770f4a3..bb23373bb4 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -22,6 +22,7 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.io.InterruptedIOException; +import java.io.UncheckedIOException; import java.net.URI; import java.nio.file.AccessDeniedException; import java.text.DateFormat; @@ -91,6 +92,7 @@ import org.apache.hadoop.fs.CreateFlag; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FSDataOutputStreamBuilder; import org.apache.hadoop.fs.Globber; import org.apache.hadoop.fs.Options; import org.apache.hadoop.fs.impl.OpenFileParameters; @@ -104,6 +106,7 @@ import org.apache.hadoop.fs.s3a.impl.ContextAccessors; import org.apache.hadoop.fs.s3a.impl.CopyFromLocalOperation; import org.apache.hadoop.fs.s3a.impl.CopyOutcome; +import org.apache.hadoop.fs.s3a.impl.CreateFileBuilder; import org.apache.hadoop.fs.s3a.impl.DeleteOperation; import org.apache.hadoop.fs.s3a.impl.DirectoryPolicy; import org.apache.hadoop.fs.s3a.impl.DirectoryPolicyImpl; @@ -114,6 +117,7 @@ import org.apache.hadoop.fs.s3a.impl.MkdirOperation; import org.apache.hadoop.fs.s3a.impl.OpenFileSupport; import org.apache.hadoop.fs.s3a.impl.OperationCallbacks; +import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; import org.apache.hadoop.fs.s3a.impl.RenameOperation; import org.apache.hadoop.fs.s3a.impl.RequestFactoryImpl; import org.apache.hadoop.fs.s3a.impl.S3AMultipartUploaderBuilder; @@ -207,7 +211,8 @@ import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_ABORT_PENDING_UPLOADS; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_STAGING_ABORT_PENDING_UPLOADS; import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit; -import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletionIgnoringExceptions; +import static org.apache.hadoop.fs.s3a.impl.CreateFileBuilder.OPTIONS_CREATE_FILE_NO_OVERWRITE; +import static org.apache.hadoop.fs.s3a.impl.CreateFileBuilder.OPTIONS_CREATE_FILE_OVERWRITE; import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isObjectNotFound; import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isUnknownBucket; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.AP_INACCESSIBLE; @@ -1614,10 +1619,15 @@ public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite, int bufferSize, short replication, long blockSize, Progressable progress) throws IOException { final Path path = qualify(f); + // the span will be picked up inside the output stream return trackDurationAndSpan(INVOCATION_CREATE, path, () -> - innerCreateFile(path, permission, overwrite, bufferSize, replication, - blockSize, progress)); + innerCreateFile(path, + progress, + getActiveAuditSpan(), + overwrite + ? OPTIONS_CREATE_FILE_OVERWRITE + : OPTIONS_CREATE_FILE_NO_OVERWRITE)); } /** @@ -1625,58 +1635,68 @@ public FSDataOutputStream create(Path f, FsPermission permission, * reporting; in the active span. * Retry policy: retrying, translated on the getFileStatus() probe. * No data is uploaded to S3 in this call, so no retry issues related to that. + * The "performance" flag disables safety checks for the path being a file, + * parent directory existing, and doesn't attempt to delete + * dir markers, irrespective of FS settings. + * If true, this method call does no IO at all. * @param path the file name to open - * @param permission the permission to set. - * @param overwrite if a file with this name already exists, then if true, - * the file will be overwritten, and if false an error will be thrown. - * @param bufferSize the size of the buffer to be used. - * @param replication required block replication for the file. - * @param blockSize the requested block size. * @param progress the progress reporter. + * @param auditSpan audit span + * @param options options for the file * @throws IOException in the event of IO related errors. - * @see #setPermission(Path, FsPermission) */ @SuppressWarnings("IOResourceOpenedButNotSafelyClosed") @Retries.RetryTranslated - private FSDataOutputStream innerCreateFile(Path path, - FsPermission permission, - boolean overwrite, - int bufferSize, - short replication, - long blockSize, - Progressable progress) throws IOException { + private FSDataOutputStream innerCreateFile( + final Path path, + final Progressable progress, + final AuditSpan auditSpan, + final CreateFileBuilder.CreateFileOptions options) throws IOException { + auditSpan.activate(); String key = pathToKey(path); - FileStatus status = null; - try { - // get the status or throw an FNFE. - // when overwriting, there is no need to look for any existing file, - // and attempting to do so can poison the load balancers with 404 - // entries. - status = innerGetFileStatus(path, false, - overwrite - ? StatusProbeEnum.DIRECTORIES - : StatusProbeEnum.ALL); + EnumSet flags = options.getFlags(); + boolean overwrite = flags.contains(CreateFlag.OVERWRITE); + boolean performance = options.isPerformance(); + boolean skipProbes = performance || isUnderMagicCommitPath(path); + if (skipProbes) { + LOG.debug("Skipping existence/overwrite checks"); + } else { + try { + // get the status or throw an FNFE. + // when overwriting, there is no need to look for any existing file, + // just a directory (for safety) + FileStatus status = innerGetFileStatus(path, false, + overwrite + ? StatusProbeEnum.DIRECTORIES + : StatusProbeEnum.ALL); - // if the thread reaches here, there is something at the path - if (status.isDirectory()) { - // path references a directory: automatic error - throw new FileAlreadyExistsException(path + " is a directory"); + // if the thread reaches here, there is something at the path + if (status.isDirectory()) { + // path references a directory: automatic error + throw new FileAlreadyExistsException(path + " is a directory"); + } + if (!overwrite) { + // path references a file and overwrite is disabled + throw new FileAlreadyExistsException(path + " already exists"); + } + LOG.debug("Overwriting file {}", path); + } catch (FileNotFoundException e) { + // this means there is nothing at the path; all good. } - if (!overwrite) { - // path references a file and overwrite is disabled - throw new FileAlreadyExistsException(path + " already exists"); - } - LOG.debug("Overwriting file {}", path); - } catch (FileNotFoundException e) { - // this means the file is not found - } instrumentation.fileCreated(); - PutTracker putTracker = - committerIntegration.createTracker(path, key); - String destKey = putTracker.getDestKey(); final BlockOutputStreamStatistics outputStreamStatistics = statisticsContext.newOutputStreamStatistics(); + PutTracker putTracker = + committerIntegration.createTracker(path, key, outputStreamStatistics); + String destKey = putTracker.getDestKey(); + + // put options are derived from the path and the + // option builder. + boolean keep = performance || keepDirectoryMarkers(path); + final PutObjectOptions putOptions = + new PutObjectOptions(keep, null, options.getHeaders()); + final S3ABlockOutputStream.BlockOutputStreamBuilder builder = S3ABlockOutputStream.builder() .withKey(destKey) @@ -1686,7 +1706,7 @@ private FSDataOutputStream innerCreateFile(Path path, .withProgress(progress) .withPutTracker(putTracker) .withWriteOperations( - createWriteOperationHelper(getActiveAuditSpan())) + createWriteOperationHelper(auditSpan)) .withExecutorService( new SemaphoredDelegatingExecutor( boundedThreadPool, @@ -1697,12 +1717,12 @@ private FSDataOutputStream innerCreateFile(Path path, getConf().getBoolean( DOWNGRADE_SYNCABLE_EXCEPTIONS, DOWNGRADE_SYNCABLE_EXCEPTIONS_DEFAULT)) - .withCSEEnabled(isCSEEnabled); + .withCSEEnabled(isCSEEnabled) + .withPutOptions(putOptions); return new FSDataOutputStream( new S3ABlockOutputStream(builder), null); } - /** * Create a Write Operation Helper with the current active span. * All operations made through this helper will activate the @@ -1735,10 +1755,65 @@ public WriteOperationHelper createWriteOperationHelper(AuditSpan auditSpan) { auditSpan); } + /** + * Create instance of an FSDataOutputStreamBuilder for + * creating a file at the given path. + * @param path path to create + * @return a builder. + * @throws UncheckedIOException for problems creating the audit span + */ + @Override + @AuditEntryPoint + public FSDataOutputStreamBuilder createFile(final Path path) { + try { + final Path qualified = qualify(path); + final AuditSpan span = entryPoint(INVOCATION_CREATE_FILE, + pathToKey(qualified), + null); + return new CreateFileBuilder(this, + qualified, + new CreateFileBuilderCallbacksImpl(INVOCATION_CREATE_FILE, span)) + .create() + .overwrite(true); + } catch (IOException e) { + // catch any IOEs raised in span creation and convert to + // an UncheckedIOException + throw new UncheckedIOException(e); + } + } + + /** + * Callback for create file operations. + */ + private final class CreateFileBuilderCallbacksImpl implements + CreateFileBuilder.CreateFileBuilderCallbacks { + + private final Statistic statistic; + /** span for operations. */ + private final AuditSpan span; + + private CreateFileBuilderCallbacksImpl( + final Statistic statistic, + final AuditSpan span) { + this.statistic = statistic; + this.span = span; + } + + @Override + public FSDataOutputStream createFileFromBuilder( + final Path path, + final Progressable progress, + final CreateFileBuilder.CreateFileOptions options) throws IOException { + // the span will be picked up inside the output stream + return trackDuration(getDurationTrackerFactory(), statistic.getSymbol(), () -> + innerCreateFile(path, progress, span, options)); + } + } + /** * {@inheritDoc} - * @throws FileNotFoundException if the parent directory is not present -or - * is not a directory. + * The S3A implementations downgrades to the recursive creation, to avoid + * any race conditions with parent entries "disappearing". */ @Override @AuditEntryPoint @@ -1750,30 +1825,23 @@ public FSDataOutputStream createNonRecursive(Path p, long blockSize, Progressable progress) throws IOException { final Path path = makeQualified(p); - // this span is passed into the stream. - try (AuditSpan span = entryPoint(INVOCATION_CREATE_NON_RECURSIVE, path)) { - Path parent = path.getParent(); - // expect this to raise an exception if there is no parent dir - if (parent != null && !parent.isRoot()) { - S3AFileStatus status; - try { - // optimize for the directory existing: Call list first - status = innerGetFileStatus(parent, false, - StatusProbeEnum.DIRECTORIES); - } catch (FileNotFoundException e) { - // no dir, fall back to looking for a file - // (failure condition if true) - status = innerGetFileStatus(parent, false, - StatusProbeEnum.HEAD_ONLY); - } - if (!status.isDirectory()) { - throw new FileAlreadyExistsException("Not a directory: " + parent); - } - } - return innerCreateFile(path, permission, - flags.contains(CreateFlag.OVERWRITE), bufferSize, - replication, blockSize, progress); + + // span is created and passed in to the callbacks. + final AuditSpan span = entryPoint(INVOCATION_CREATE_NON_RECURSIVE, + pathToKey(path), + null); + // uses the CreateFileBuilder, filling it in with the relevant arguments. + final CreateFileBuilder builder = new CreateFileBuilder(this, + path, + new CreateFileBuilderCallbacksImpl(INVOCATION_CREATE_NON_RECURSIVE, span)) + .create() + .withFlags(flags) + .blockSize(blockSize) + .bufferSize(bufferSize); + if (progress != null) { + builder.progress(progress); } + return builder.build(); } /** @@ -2671,7 +2739,7 @@ private DeleteObjectsResult deleteObjects(DeleteObjectsRequest deleteRequest) */ public PutObjectRequest newPutObjectRequest(String key, ObjectMetadata metadata, File srcfile) { - return requestFactory.newPutObjectRequest(key, metadata, srcfile); + return requestFactory.newPutObjectRequest(key, metadata, null, srcfile); } /** @@ -2721,12 +2789,14 @@ public UploadInfo putObject(PutObjectRequest putObjectRequest) { * Auditing: must be inside an audit span. * Important: this call will close any input stream in the request. * @param putObjectRequest the request + * @param putOptions put object options * @return the upload initiated * @throws AmazonClientException on problems */ @VisibleForTesting - @Retries.OnceRaw("For PUT; post-PUT actions are RetryTranslated") - PutObjectResult putObjectDirect(PutObjectRequest putObjectRequest) + @Retries.OnceRaw("For PUT; post-PUT actions are RetryExceptionsSwallowed") + PutObjectResult putObjectDirect(PutObjectRequest putObjectRequest, + PutObjectOptions putOptions) throws AmazonClientException { long len = getPutRequestLength(putObjectRequest); LOG.debug("PUT {} bytes to {}", len, putObjectRequest.getKey()); @@ -2737,9 +2807,10 @@ PutObjectResult putObjectDirect(PutObjectRequest putObjectRequest) OBJECT_PUT_REQUESTS.getSymbol(), () -> s3.putObject(putObjectRequest)); incrementPutCompletedStatistics(true, len); - // update metadata + // apply any post-write actions. finishedWrite(putObjectRequest.getKey(), len, - result.getETag(), result.getVersionId()); + result.getETag(), result.getVersionId(), + putOptions); return result; } catch (SdkBaseException e) { incrementPutCompletedStatistics(false, len); @@ -3011,7 +3082,7 @@ private void createFakeDirectoryIfNecessary(Path f) // is mostly harmless to create a new one. if (!key.isEmpty() && !s3Exists(f, StatusProbeEnum.DIRECTORIES)) { LOG.debug("Creating new fake directory at {}", f); - createFakeDirectory(key); + createFakeDirectory(key, putOptionsForPath(f)); } } @@ -3026,7 +3097,7 @@ private void createFakeDirectoryIfNecessary(Path f) protected void maybeCreateFakeParentDirectory(Path path) throws IOException, AmazonClientException { Path parent = path.getParent(); - if (parent != null && !parent.isRoot()) { + if (parent != null && !parent.isRoot() && !isUnderMagicCommitPath(parent)) { createFakeDirectoryIfNecessary(parent); } } @@ -3197,6 +3268,11 @@ public UserGroupInformation getOwner() { * Make the given path and all non-existent parents into * directories. Has the semantics of Unix {@code 'mkdir -p'}. * Existence of the directory hierarchy is not an error. + * Parent elements are scanned to see if any are a file, + * except under __magic paths. + * There the FS assumes that the destination directory creation + * did that scan and that paths in job/task attempts are all + * "well formed" * @param p path to create * @param permission to apply to path * @return true if a directory was created or already existed @@ -3214,7 +3290,8 @@ public boolean mkdirs(Path p, FsPermission permission) throws IOException, new MkdirOperation( createStoreContext(), path, - createMkdirOperationCallbacks())); + createMkdirOperationCallbacks(), + isMagicCommitPath(path))); } /** @@ -3240,9 +3317,13 @@ public S3AFileStatus probePathStatus(final Path path, } @Override - public void createFakeDirectory(final String key) + public void createFakeDirectory(final Path dir, final boolean keepMarkers) throws IOException { - S3AFileSystem.this.createEmptyObject(key); + S3AFileSystem.this.createFakeDirectory( + pathToKey(dir), + keepMarkers + ? PutObjectOptions.keepingDirs() + : putOptionsForPath(dir)); } } @@ -3608,7 +3689,7 @@ public void copyLocalFileFromTo(File file, Path from, Path to) throws IOExceptio S3AFileSystem.this.invoker.retry( "putObject(" + "" + ")", to.toString(), true, - () -> executePut(putObjectRequest, progress)); + () -> executePut(putObjectRequest, progress, putOptionsForPath(to))); return null; }); @@ -3627,7 +3708,7 @@ public boolean createEmptyDir(Path path, StoreContext storeContext) new MkdirOperation( storeContext, path, - createMkdirOperationCallbacks())); + createMkdirOperationCallbacks(), false)); } } @@ -3637,14 +3718,18 @@ public boolean createEmptyDir(Path path, StoreContext storeContext) * aborted before an {@code InterruptedIOException} is thrown. * @param putObjectRequest request * @param progress optional progress callback + * @param putOptions put object options * @return the upload result * @throws InterruptedIOException if the blocking was interrupted. */ - @Retries.OnceRaw("For PUT; post-PUT actions are RetryTranslated") - UploadResult executePut(PutObjectRequest putObjectRequest, - Progressable progress) + @Retries.OnceRaw("For PUT; post-PUT actions are RetrySwallowed") + UploadResult executePut( + final PutObjectRequest putObjectRequest, + final Progressable progress, + final PutObjectOptions putOptions) throws InterruptedIOException { String key = putObjectRequest.getKey(); + long len = getPutRequestLength(putObjectRequest); UploadInfo info = putObject(putObjectRequest); Upload upload = info.getUpload(); ProgressableProgressListener listener = new ProgressableProgressListener( @@ -3652,9 +3737,10 @@ UploadResult executePut(PutObjectRequest putObjectRequest, upload.addProgressListener(listener); UploadResult result = waitForUploadCompletion(key, info); listener.uploadCompleted(); + // post-write actions - finishedWrite(key, info.getLength(), - result.getETag(), result.getVersionId()); + finishedWrite(key, len, + result.getETag(), result.getVersionId(), putOptions); return result; } @@ -3663,7 +3749,9 @@ UploadResult executePut(PutObjectRequest putObjectRequest, * If the waiting for completion is interrupted, the upload will be * aborted before an {@code InterruptedIOException} is thrown. * If the upload (or its result collection) failed, this is where - * the failure is raised as an AWS exception + * the failure is raised as an AWS exception. + * Calls {@link #incrementPutCompletedStatistics(boolean, long)} + * to update the statistics. * @param key destination key * @param uploadInfo upload to wait for * @return the upload result @@ -3985,63 +4073,64 @@ InitiateMultipartUploadResult initiateMultipartUpload( /** * Perform post-write actions. - *

+ *

* This operation MUST be called after any PUT/multipart PUT completes * successfully. - *

- * The actions include: - *
    - *
  1. - * Calling - * {@link #deleteUnnecessaryFakeDirectories(Path)} - * if directory markers are not being retained. - *
  2. - *
  3. - * Updating any metadata store with details on the newly created - * object. - *
  4. - *
+ *

+ * The actions include calling + * {@link #deleteUnnecessaryFakeDirectories(Path)} + * if directory markers are not being retained. * @param key key written to * @param length total length of file written * @param eTag eTag of the written object * @param versionId S3 object versionId of the written object + * @param putOptions put object options */ @InterfaceAudience.Private - @Retries.RetryTranslated("Except if failOnMetadataWriteError=false, in which" - + " case RetryExceptionsSwallowed") - void finishedWrite(String key, long length, String eTag, String versionId) { + @Retries.RetryExceptionsSwallowed + void finishedWrite( + String key, + long length, + String eTag, + String versionId, + PutObjectOptions putOptions) { LOG.debug("Finished write to {}, len {}. etag {}, version {}", key, length, eTag, versionId); - Path p = keyToQualifiedPath(key); Preconditions.checkArgument(length >= 0, "content length is negative"); - // kick off an async delete - CompletableFuture deletion; - if (!keepDirectoryMarkers(p)) { - deletion = submit( - unboundedThreadPool, getActiveAuditSpan(), - () -> { - deleteUnnecessaryFakeDirectories( - p.getParent() - ); - return null; - }); - } else { - deletion = null; + if (!putOptions.isKeepMarkers()) { + Path p = keyToQualifiedPath(key); + deleteUnnecessaryFakeDirectories(p.getParent()); } - - // catch up with any delete operation. - waitForCompletionIgnoringExceptions(deletion); } /** * Should we keep directory markers under the path being created * by mkdir/file creation/rename? + * This is done if marker retention is enabled for the path, + * or it is under a magic path where we are saving IOPs + * knowing that all committers are on the same code version and + * therefore marker aware. * @param path path to probe * @return true if the markers MAY be retained, * false if they MUST be deleted */ private boolean keepDirectoryMarkers(Path path) { - return directoryPolicy.keepDirectoryMarkers(path); + return directoryPolicy.keepDirectoryMarkers(path) + || isUnderMagicCommitPath(path); + } + + /** + * Should we keep directory markers under the path being created + * by mkdir/file creation/rename? + * See {@link #keepDirectoryMarkers(Path)} for the policy. + * + * @param path path to probe + * @return the options to use with the put request + */ + private PutObjectOptions putOptionsForPath(Path path) { + return keepDirectoryMarkers(path) + ? PutObjectOptions.keepingDirs() + : PutObjectOptions.deletingDirs(); } /** @@ -4078,27 +4167,32 @@ private void deleteUnnecessaryFakeDirectories(Path path) { * Create a fake directory, always ending in "/". * Retry policy: retrying; translated. * @param objectName name of directory object. + * @param putOptions put object options * @throws IOException IO failure */ @Retries.RetryTranslated - private void createFakeDirectory(final String objectName) + private void createFakeDirectory(final String objectName, + final PutObjectOptions putOptions) throws IOException { - createEmptyObject(objectName); + createEmptyObject(objectName, putOptions); } /** * Used to create an empty file that represents an empty directory. + * The policy for deleting parent dirs depends on the path, dir + * status and the putOptions value. * Retry policy: retrying; translated. * @param objectName object to create + * @param putOptions put object options * @throws IOException IO failure */ @Retries.RetryTranslated - private void createEmptyObject(final String objectName) + private void createEmptyObject(final String objectName, PutObjectOptions putOptions) throws IOException { invoker.retry("PUT 0-byte object ", objectName, true, () -> putObjectDirect(getRequestFactory() - .newDirectoryMarkerRequest(objectName))); + .newDirectoryMarkerRequest(objectName), putOptions)); incrementPutProgressStatistics(objectName, 0); instrumentation.directoryCreated(); } @@ -4207,14 +4301,26 @@ public boolean isMagicCommitEnabled() { /** * Predicate: is a path a magic commit path? - * True if magic commit is enabled and the path qualifies as special. + * True if magic commit is enabled and the path qualifies as special, + * and is not a a .pending or .pendingset file, * @param path path to examine - * @return true if the path is or is under a magic directory + * @return true if writing a file to the path triggers a "magic" write. */ public boolean isMagicCommitPath(Path path) { return committerIntegration.isMagicCommitPath(path); } + /** + * Predicate: is a path under a magic commit path? + * True if magic commit is enabled and the path is under __magic, + * irrespective of file type. + * @param path path to examine + * @return true if the path is in a magic dir and the FS has magic writes enabled. + */ + private boolean isUnderMagicCommitPath(Path path) { + return committerIntegration.isUnderMagicPath(path); + } + /** * Increments the statistic {@link Statistic#INVOCATION_GLOB_STATUS}. * Override superclass so as to disable symlink resolution as symlinks @@ -4766,10 +4872,20 @@ public boolean hasPathCapability(final Path path, final String capability) case STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_KEEP: case STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_DELETE: case STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_AUTHORITATIVE: - case STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_KEEP: - case STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_DELETE: return getDirectoryMarkerPolicy().hasPathCapability(path, cap); + // keep for a magic path or if the policy retains it + case STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_KEEP: + return keepDirectoryMarkers(path); + // delete is the opposite of keep + case STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_DELETE: + return !keepDirectoryMarkers(path); + + // create file options + case FS_S3A_CREATE_PERFORMANCE: + case FS_S3A_CREATE_HEADER: + return true; + default: return super.hasPathCapability(p, cap); } 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 eec6366720..67734b7502 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 @@ -1308,8 +1308,18 @@ private void mergeOutputStreamStatistics( incrementCounter(STREAM_WRITE_EXCEPTIONS, source.lookupCounterValue( StreamStatisticNames.STREAM_WRITE_EXCEPTIONS)); + // merge in all the IOStatistics - this.getIOStatistics().aggregate(source.getIOStatistics()); + final IOStatisticsStore sourceIOStatistics = source.getIOStatistics(); + this.getIOStatistics().aggregate(sourceIOStatistics); + + // propagate any extra values into the FS-level stats. + incrementMutableCounter(OBJECT_PUT_REQUESTS.getSymbol(), + sourceIOStatistics.counters().get(OBJECT_PUT_REQUESTS.getSymbol())); + incrementMutableCounter( + COMMITTER_MAGIC_MARKER_PUT.getSymbol(), + sourceIOStatistics.counters().get(COMMITTER_MAGIC_MARKER_PUT.getSymbol())); + } /** @@ -1366,9 +1376,12 @@ private OutputStreamStatistics( STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING.getSymbol()) .withDurationTracking( ACTION_EXECUTOR_ACQUIRED, + COMMITTER_MAGIC_MARKER_PUT.getSymbol(), INVOCATION_ABORT.getSymbol(), + MULTIPART_UPLOAD_COMPLETED.getSymbol(), OBJECT_MULTIPART_UPLOAD_ABORTED.getSymbol(), - MULTIPART_UPLOAD_COMPLETED.getSymbol()) + OBJECT_MULTIPART_UPLOAD_INITIATED.getSymbol(), + OBJECT_PUT_REQUESTS.getSymbol()) .build(); setIOStatistics(st); // these are extracted to avoid lookups on heavily used counters. @@ -1630,6 +1643,7 @@ private CommitterStatisticsImpl() { COMMITTER_TASKS_SUCCEEDED.getSymbol()) .withDurationTracking( COMMITTER_COMMIT_JOB.getSymbol(), + COMMITTER_LOAD_SINGLE_PENDING_FILE.getSymbol(), COMMITTER_MATERIALIZE_FILE.getSymbol(), COMMITTER_STAGE_FILE_UPLOAD.getSymbol()) .build(); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java index d644d3f476..e7e741d42c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java @@ -88,6 +88,7 @@ import static org.apache.hadoop.fs.s3a.impl.InternalConstants.CSE_PADDING_LENGTH; import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.translateDeleteException; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; +import static org.apache.hadoop.util.functional.RemoteIterators.filteringRemoteIterator; /** * Utility methods for S3A code. @@ -1467,17 +1468,22 @@ public static List flatmapLocatedFiles( /** * List located files and filter them as a classic listFiles(path, filter) * would do. + * This will be incremental, fetching pages async. + * While it is rare for job to have many thousands of files, jobs + * against versioned buckets may return earlier if there are many + * non-visible objects. * @param fileSystem filesystem * @param path path to list * @param recursive recursive listing? * @param filter filter for the filename - * @return the filtered list of entries + * @return interator over the entries. * @throws IOException IO failure. */ - public static List listAndFilter(FileSystem fileSystem, + public static RemoteIterator listAndFilter(FileSystem fileSystem, Path path, boolean recursive, PathFilter filter) throws IOException { - return flatmapLocatedFiles(fileSystem.listFiles(path, recursive), - status -> maybe(filter.accept(status.getPath()), status)); + return filteringRemoteIterator( + fileSystem.listFiles(path, recursive), + status -> filter.accept(status.getPath())); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java index 86cb18076c..dfe9fdf2d8 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java @@ -111,6 +111,10 @@ public enum Statistic { StoreStatisticNames.OP_CREATE, "Calls of create()", TYPE_DURATION), + INVOCATION_CREATE_FILE( + StoreStatisticNames.OP_CREATE_FILE, + "Calls of createFile()", + TYPE_DURATION), INVOCATION_CREATE_NON_RECURSIVE( StoreStatisticNames.OP_CREATE_NON_RECURSIVE, "Calls of createNonRecursive()", @@ -459,10 +463,19 @@ public enum Statistic { "committer_commits_reverted", "Count of commits reverted", TYPE_COUNTER), + COMMITTER_LOAD_SINGLE_PENDING_FILE( + "committer_load_single_pending_file", + "Duration to load a single pending file in task commit", + TYPE_DURATION), COMMITTER_MAGIC_FILES_CREATED( "committer_magic_files_created", "Count of files created under 'magic' paths", TYPE_COUNTER), + + COMMITTER_MAGIC_MARKER_PUT( + "committer_magic_marker_put", + "Duration Tracking of marker files created under 'magic' paths", + TYPE_DURATION), COMMITTER_MATERIALIZE_FILE( "committer_materialize_file", "Duration Tracking of time to materialize a file in job commit", diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java index ee91bacfb0..ce50f0b85e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java @@ -24,7 +24,6 @@ import java.io.IOException; import java.io.InputStream; import java.util.List; -import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; import com.amazonaws.services.s3.model.AmazonS3Exception; @@ -40,8 +39,6 @@ import com.amazonaws.services.s3.model.SelectObjectContentResult; import com.amazonaws.services.s3.model.UploadPartRequest; import com.amazonaws.services.s3.model.UploadPartResult; -import com.amazonaws.services.s3.transfer.model.UploadResult; -import org.apache.hadoop.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,6 +48,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; import org.apache.hadoop.fs.s3a.impl.StoreContext; import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; import org.apache.hadoop.fs.s3a.select.SelectBinding; @@ -58,6 +56,7 @@ import org.apache.hadoop.fs.store.audit.AuditSpanSource; import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.functional.CallableRaisingIOE; +import org.apache.hadoop.util.Preconditions; import static org.apache.hadoop.util.Preconditions.checkNotNull; import static org.apache.hadoop.fs.s3a.Invoker.*; @@ -234,22 +233,20 @@ private void deactivateAuditSpan() { * @param destKey destination key * @param inputStream source data. * @param length size, if known. Use -1 for not known - * @param headers optional map of custom headers. + * @param options options for the request * @return the request */ @Retries.OnceRaw public PutObjectRequest createPutObjectRequest(String destKey, InputStream inputStream, long length, - final Map headers) { + final PutObjectOptions options) { activateAuditSpan(); ObjectMetadata objectMetadata = newObjectMetadata(length); - if (headers != null) { - objectMetadata.setUserMetadata(headers); - } return getRequestFactory().newPutObjectRequest( destKey, objectMetadata, + options, inputStream); } @@ -257,18 +254,26 @@ public PutObjectRequest createPutObjectRequest(String destKey, * Create a {@link PutObjectRequest} request to upload a file. * @param dest key to PUT to. * @param sourceFile source file + * @param options options for the request * @return the request */ @Retries.OnceRaw - public PutObjectRequest createPutObjectRequest(String dest, - File sourceFile) { + public PutObjectRequest createPutObjectRequest( + String dest, + File sourceFile, + final PutObjectOptions options) { Preconditions.checkState(sourceFile.length() < Integer.MAX_VALUE, "File length is too big for a single PUT upload"); activateAuditSpan(); - return getRequestFactory(). + final ObjectMetadata objectMetadata = + newObjectMetadata((int) sourceFile.length()); + + PutObjectRequest putObjectRequest = getRequestFactory(). newPutObjectRequest(dest, - newObjectMetadata((int) sourceFile.length()), + objectMetadata, + options, sourceFile); + return putObjectRequest; } /** @@ -298,21 +303,20 @@ public ObjectMetadata newObjectMetadata(long length) { } /** - * Start the multipart upload process. - * Retry policy: retrying, translated. - * @param destKey destination of upload - * @return the upload result containing the ID - * @throws IOException IO problem + * {@inheritDoc} */ @Retries.RetryTranslated - public String initiateMultiPartUpload(String destKey) throws IOException { + public String initiateMultiPartUpload( + final String destKey, + final PutObjectOptions options) + throws IOException { LOG.debug("Initiating Multipart upload to {}", destKey); try (AuditSpan span = activateAuditSpan()) { return retry("initiate MultiPartUpload", destKey, true, () -> { final InitiateMultipartUploadRequest initiateMPURequest = getRequestFactory().newMultipartUploadRequest( - destKey); + destKey, options); return owner.initiateMultipartUpload(initiateMPURequest) .getUploadId(); }); @@ -322,13 +326,14 @@ public String initiateMultiPartUpload(String destKey) throws IOException { /** * Finalize a multipart PUT operation. * This completes the upload, and, if that works, calls - * {@link S3AFileSystem#finishedWrite(String, long, String, String)} + * {@link S3AFileSystem#finishedWrite(String, long, String, String, org.apache.hadoop.fs.s3a.impl.PutObjectOptions)} * to update the filesystem. * Retry policy: retrying, translated. * @param destKey destination of the commit * @param uploadId multipart operation Id * @param partETags list of partial uploads * @param length length of the upload + * @param putOptions put object options * @param retrying retrying callback * @return the result of the operation. * @throws IOException on problems. @@ -339,6 +344,7 @@ private CompleteMultipartUploadResult finalizeMultipartUpload( String uploadId, List partETags, long length, + PutObjectOptions putOptions, Retried retrying) throws IOException { if (partETags.isEmpty()) { throw new PathIOException(destKey, @@ -357,7 +363,8 @@ private CompleteMultipartUploadResult finalizeMultipartUpload( request); }); owner.finishedWrite(destKey, length, uploadResult.getETag(), - uploadResult.getVersionId()); + uploadResult.getVersionId(), + putOptions); return uploadResult; } } @@ -373,6 +380,7 @@ private CompleteMultipartUploadResult finalizeMultipartUpload( * @param length length of the upload * @param errorCount a counter incremented by 1 on every error; for * use in statistics + * @param putOptions put object options * @return the result of the operation. * @throws IOException if problems arose which could not be retried, or * the retry count was exceeded @@ -383,7 +391,8 @@ public CompleteMultipartUploadResult completeMPUwithRetries( String uploadId, List partETags, long length, - AtomicInteger errorCount) + AtomicInteger errorCount, + PutObjectOptions putOptions) throws IOException { checkNotNull(uploadId); checkNotNull(partETags); @@ -393,8 +402,8 @@ public CompleteMultipartUploadResult completeMPUwithRetries( uploadId, partETags, length, - (text, e, r, i) -> errorCount.incrementAndGet() - ); + putOptions, + (text, e, r, i) -> errorCount.incrementAndGet()); } /** @@ -550,37 +559,43 @@ public String toString() { * Byte length is calculated from the file length, or, if there is no * file, from the content length of the header. * @param putObjectRequest the request + * @param putOptions put object options * @return the upload initiated * @throws IOException on problems */ @Retries.RetryTranslated - public PutObjectResult putObject(PutObjectRequest putObjectRequest) + public PutObjectResult putObject(PutObjectRequest putObjectRequest, + PutObjectOptions putOptions) throws IOException { return retry("Writing Object", putObjectRequest.getKey(), true, withinAuditSpan(getAuditSpan(), () -> - owner.putObjectDirect(putObjectRequest))); + owner.putObjectDirect(putObjectRequest, putOptions))); } /** - * PUT an object via the transfer manager. + * PUT an object. + * * @param putObjectRequest the request - * @return the result of the operation + * @param putOptions put object options + * * @throws IOException on problems */ @Retries.RetryTranslated - public UploadResult uploadObject(PutObjectRequest putObjectRequest) + public void uploadObject(PutObjectRequest putObjectRequest, + PutObjectOptions putOptions) throws IOException { - // no retry; rely on xfer manager logic - return retry("Writing Object", + + retry("Writing Object", putObjectRequest.getKey(), true, withinAuditSpan(getAuditSpan(), () -> - owner.executePut(putObjectRequest, null))); + owner.putObjectDirect(putObjectRequest, putOptions))); } /** * Revert a commit by deleting the file. - * Relies on retry code in filesystem + * Relies on retry code in filesystem. + * Does not attempt to recreate the parent directory * @throws IOException on problems * @param destKey destination key */ @@ -591,13 +606,14 @@ public void revertCommit(String destKey) throws IOException { Path destPath = owner.keyToQualifiedPath(destKey); owner.deleteObjectAtPath(destPath, destKey, true); - owner.maybeCreateFakeParentDirectory(destPath); })); } /** * This completes a multipart upload to the destination key via * {@code finalizeMultipartUpload()}. + * Markers are never deleted on commit; this avoids having to + * issue many duplicate deletions. * Retry policy: retrying, translated. * Retries increment the {@code errorCount} counter. * @param destKey destination @@ -623,8 +639,8 @@ public CompleteMultipartUploadResult commitUpload( uploadId, partETags, length, - Invoker.NO_OP - ); + PutObjectOptions.keepingDirs(), + Invoker.NO_OP); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java index 7604bbe46b..321390446f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java @@ -25,7 +25,6 @@ import java.io.IOException; import java.io.InputStream; import java.util.List; -import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; import com.amazonaws.services.s3.model.CompleteMultipartUploadResult; @@ -38,11 +37,11 @@ import com.amazonaws.services.s3.model.SelectObjectContentResult; import com.amazonaws.services.s3.model.UploadPartRequest; import com.amazonaws.services.s3.model.UploadPartResult; -import com.amazonaws.services.s3.transfer.model.UploadResult; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; import org.apache.hadoop.fs.store.audit.AuditSpanSource; import org.apache.hadoop.util.functional.CallableRaisingIOE; @@ -79,22 +78,25 @@ T retry(String action, * @param destKey destination key * @param inputStream source data. * @param length size, if known. Use -1 for not known - * @param headers optional map of custom headers. + * @param options options for the request * @return the request */ PutObjectRequest createPutObjectRequest(String destKey, InputStream inputStream, long length, - @Nullable Map headers); + @Nullable PutObjectOptions options); /** * Create a {@link PutObjectRequest} request to upload a file. * @param dest key to PUT to. * @param sourceFile source file + * @param options options for the request * @return the request */ - PutObjectRequest createPutObjectRequest(String dest, - File sourceFile); + PutObjectRequest createPutObjectRequest( + String dest, + File sourceFile, + @Nullable PutObjectOptions options); /** * Callback on a successful write. @@ -121,11 +123,12 @@ PutObjectRequest createPutObjectRequest(String dest, * Start the multipart upload process. * Retry policy: retrying, translated. * @param destKey destination of upload + * @param options options for the put request * @return the upload result containing the ID * @throws IOException IO problem */ @Retries.RetryTranslated - String initiateMultiPartUpload(String destKey) throws IOException; + String initiateMultiPartUpload(String destKey, PutObjectOptions options) throws IOException; /** * This completes a multipart upload to the destination key via @@ -138,6 +141,7 @@ PutObjectRequest createPutObjectRequest(String dest, * @param length length of the upload * @param errorCount a counter incremented by 1 on every error; for * use in statistics + * @param putOptions put object options * @return the result of the operation. * @throws IOException if problems arose which could not be retried, or * the retry count was exceeded @@ -148,7 +152,8 @@ CompleteMultipartUploadResult completeMPUwithRetries( String uploadId, List partETags, long length, - AtomicInteger errorCount) + AtomicInteger errorCount, + PutObjectOptions putOptions) throws IOException; /** @@ -238,26 +243,32 @@ UploadPartRequest newUploadPartRequest( * Byte length is calculated from the file length, or, if there is no * file, from the content length of the header. * @param putObjectRequest the request + * @param putOptions put object options * @return the upload initiated * @throws IOException on problems */ @Retries.RetryTranslated - PutObjectResult putObject(PutObjectRequest putObjectRequest) + PutObjectResult putObject(PutObjectRequest putObjectRequest, + PutObjectOptions putOptions) throws IOException; /** * PUT an object via the transfer manager. + * * @param putObjectRequest the request - * @return the result of the operation + * @param putOptions put object options + * * @throws IOException on problems */ @Retries.RetryTranslated - UploadResult uploadObject(PutObjectRequest putObjectRequest) + void uploadObject(PutObjectRequest putObjectRequest, + PutObjectOptions putOptions) throws IOException; /** * Revert a commit by deleting the file. - * Relies on retry code in filesystem + * No attempt is made to probe for/recreate a parent dir marker + * Relies on retry code in filesystem. * @throws IOException on problems * @param destKey destination key */ diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java index 20cd27225a..cae4d3ef03 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java @@ -50,6 +50,7 @@ import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.s3a.S3AEncryptionMethods; import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets; +import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; /** * Factory for S3 objects. @@ -141,11 +142,12 @@ CopyObjectRequest newCopyObjectRequest(String srcKey, * Adds the ACL and metadata * @param key key of object * @param metadata metadata header + * @param options options for the request * @param srcfile source file * @return the request */ PutObjectRequest newPutObjectRequest(String key, - ObjectMetadata metadata, File srcfile); + ObjectMetadata metadata, PutObjectOptions options, File srcfile); /** * Create a {@link PutObjectRequest} request. @@ -153,11 +155,13 @@ PutObjectRequest newPutObjectRequest(String key, * operation. * @param key key of object * @param metadata metadata header + * @param options options for the request * @param inputStream source data. * @return the request */ PutObjectRequest newPutObjectRequest(String key, ObjectMetadata metadata, + PutObjectOptions options, InputStream inputStream); /** @@ -190,10 +194,12 @@ AbortMultipartUploadRequest newAbortMultipartUploadRequest( /** * Start a multipart upload. * @param destKey destination object key + * @param options options for the request * @return the request. */ InitiateMultipartUploadRequest newMultipartUploadRequest( - String destKey); + String destKey, + @Nullable PutObjectOptions options); /** * Complete a multipart upload. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java index f08d6448e4..78b687cc6f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java @@ -26,32 +26,30 @@ import java.util.Date; import java.util.List; import java.util.UUID; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; import com.amazonaws.services.s3.model.MultipartUpload; - -import org.apache.hadoop.classification.VisibleForTesting; -import org.apache.hadoop.util.Preconditions; -import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.commons.lang3.tuple.Pair; import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.audit.AuditConstants; -import org.apache.hadoop.fs.audit.CommonAuditContext; import org.apache.hadoop.fs.store.audit.AuditSpan; import org.apache.hadoop.fs.store.audit.AuditSpanSource; +import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; +import org.apache.hadoop.fs.s3a.commit.files.PersistentCommitData; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; import org.apache.hadoop.fs.s3a.commit.files.SuccessData; +import org.apache.hadoop.fs.s3a.commit.impl.AuditContextUpdater; +import org.apache.hadoop.fs.s3a.commit.impl.CommitContext; +import org.apache.hadoop.fs.s3a.commit.impl.CommitOperations; import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics; import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; @@ -62,24 +60,33 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.TaskAttemptID; import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitter; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestStoreOperations; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestStoreOperationsThroughFileSystem; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.util.DurationInfo; -import org.apache.hadoop.util.concurrent.HadoopExecutors; import org.apache.hadoop.util.functional.InvocationRaisingIOE; +import org.apache.hadoop.util.functional.TaskPool; -import static org.apache.hadoop.fs.s3a.Constants.THREAD_POOL_SHUTDOWN_DELAY_SECONDS; +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_MAXIMUM_CONNECTIONS; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_MAX_TOTAL_TASKS; +import static org.apache.hadoop.fs.s3a.Constants.MAXIMUM_CONNECTIONS; +import static org.apache.hadoop.fs.s3a.Constants.MAX_TOTAL_TASKS; import static org.apache.hadoop.fs.s3a.Invoker.ignoreIOExceptions; import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_COMMIT_JOB; import static org.apache.hadoop.fs.audit.CommonAuditContext.currentAuditContext; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; import static org.apache.hadoop.fs.s3a.commit.CommitUtils.*; -import static org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR.*; import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.E_NO_SPARK_UUID; import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.FS_S3A_COMMITTER_UUID; import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.FS_S3A_COMMITTER_UUID_SOURCE; import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.SPARK_WRITE_UUID; +import static org.apache.hadoop.fs.s3a.commit.impl.CommitUtilsWithMR.*; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DiagnosticKeys.STAGE; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.createJobSummaryFilename; +import static org.apache.hadoop.util.functional.RemoteIterators.toList; /** * Abstract base class for S3A committers; allows for any commonality @@ -136,8 +143,6 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter */ private final JobUUIDSource uuidSource; - private final CommonAuditContext commonAuditContext; - /** * Has this instance been used for job setup? * If so then it is safe for a locally generated @@ -145,11 +150,6 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter */ private boolean jobSetup; - /** - * Thread pool for task execution. - */ - private ExecutorService threadPool; - /** Underlying commit operations. */ private final CommitOperations commitOperations; @@ -203,9 +203,8 @@ protected AbstractS3ACommitter( Path outputPath, TaskAttemptContext context) throws IOException { super(outputPath, context); - Preconditions.checkArgument(outputPath != null, "null output path"); - Preconditions.checkArgument(context != null, "null job context"); - this.jobContext = context; + setOutputPath(outputPath); + this.jobContext = requireNonNull(context, "null job context"); this.role = "Task committer " + context.getTaskAttemptID(); setConf(context.getConfiguration()); Pair id = buildJobUUID( @@ -219,17 +218,20 @@ protected AbstractS3ACommitter( S3AFileSystem fs = getDestS3AFS(); // set this thread's context with the job ID. // audit spans created in this thread will pick - // up this value. - this.commonAuditContext = currentAuditContext(); - updateCommonContext(); + // up this value., including the commit operations instance + // soon to be created. + new AuditContextUpdater(jobContext) + .updateCurrentAuditContext(); + // the filesystem is the span source, always. - auditSpanSource = fs.getAuditSpanSource(); + this.auditSpanSource = fs.getAuditSpanSource(); this.createJobMarker = context.getConfiguration().getBoolean( CREATE_SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, DEFAULT_CREATE_SUCCESSFUL_JOB_DIR_MARKER); // the statistics are shared between this committer and its operations. this.committerStatistics = fs.newCommitterStatistics(); - this.commitOperations = new CommitOperations(fs, committerStatistics); + this.commitOperations = new CommitOperations(fs, committerStatistics, + outputPath.toString()); } /** @@ -267,8 +269,7 @@ public final Path getOutputPath() { * @param outputPath new value */ protected final void setOutputPath(Path outputPath) { - Preconditions.checkNotNull(outputPath, "Null output path"); - this.outputPath = outputPath; + this.outputPath = requireNonNull(outputPath, "Null output path"); } /** @@ -338,6 +339,12 @@ public Path getJobAttemptPath(JobContext context) { return getJobAttemptPath(getAppAttemptId(context)); } + /** + * Compute the path under which all job attempts will be placed. + * @return the path to store job attempt data. + */ + protected abstract Path getJobPath(); + /** * Compute the path where the output of a given job attempt will be placed. * @param appAttemptId the ID of the application attempt for this job. @@ -440,6 +447,7 @@ protected FileSystem getDestinationFS(Path out, Configuration config) protected boolean requiresDelayedCommitOutputInFileSystem() { return false; } + /** * Task recovery considered Unsupported: Warn and fail. * @param taskContext Context of the task whose output is being recovered @@ -450,7 +458,7 @@ public void recoverTask(TaskAttemptContext taskContext) throws IOException { LOG.warn("Cannot recover task {}", taskContext.getTaskAttemptID()); throw new PathCommitException(outputPath, String.format("Unable to recover task %s", - taskContext.getTaskAttemptID())); + taskContext.getTaskAttemptID())); } /** @@ -459,11 +467,16 @@ public void recoverTask(TaskAttemptContext taskContext) throws IOException { * * While the classic committers create a 0-byte file, the S3A committers * PUT up a the contents of a {@link SuccessData} file. + * * @param context job context * @param pending the pending commits + * + * @return the success data, even if the marker wasn't created + * * @throws IOException IO failure */ - protected void maybeCreateSuccessMarkerFromCommits(JobContext context, + protected SuccessData maybeCreateSuccessMarkerFromCommits( + JobContext context, ActiveCommit pending) throws IOException { List filenames = new ArrayList<>(pending.size()); // The list of committed objects in pending is size limited in @@ -472,41 +485,86 @@ protected void maybeCreateSuccessMarkerFromCommits(JobContext context, // load in all the pending statistics IOStatisticsSnapshot snapshot = new IOStatisticsSnapshot( pending.getIOStatistics()); + // and the current statistics snapshot.aggregate(getIOStatistics()); - maybeCreateSuccessMarker(context, filenames, snapshot); + return maybeCreateSuccessMarker(context, filenames, snapshot); } /** * if the job requires a success marker on a successful job, - * create the file {@link CommitConstants#_SUCCESS}. + * create the {@code _SUCCESS} file. * * While the classic committers create a 0-byte file, the S3A committers * PUT up a the contents of a {@link SuccessData} file. + * The file is returned, even if no marker is created. + * This is so it can be saved to a report directory. * @param context job context * @param filenames list of filenames. * @param ioStatistics any IO Statistics to include * @throws IOException IO failure + * @return the success data. */ - protected void maybeCreateSuccessMarker(JobContext context, - List filenames, + protected SuccessData maybeCreateSuccessMarker( + final JobContext context, + final List filenames, final IOStatisticsSnapshot ioStatistics) throws IOException { + + SuccessData successData = + createSuccessData(context, filenames, ioStatistics, + getDestFS().getConf()); if (createJobMarker) { - // create a success data structure and then save it - SuccessData successData = new SuccessData(); - successData.setCommitter(getName()); - successData.setJobId(uuid); - successData.setJobIdSource(uuidSource.getText()); - successData.setDescription(getRole()); - successData.setHostname(NetUtils.getLocalHostname()); - Date now = new Date(); - successData.setTimestamp(now.getTime()); - successData.setDate(now.toString()); - successData.setFilenames(filenames); - successData.getIOStatistics().aggregate(ioStatistics); + // save it to the job dest dir commitOperations.createSuccessMarker(getOutputPath(), successData, true); } + return successData; + } + + /** + * Create the success data structure from a job context. + * @param context job context. + * @param filenames short list of filenames; nullable + * @param ioStatistics IOStatistics snapshot + * @param destConf config of the dest fs, can be null + * @return the structure + * + */ + private SuccessData createSuccessData(final JobContext context, + final List filenames, + final IOStatisticsSnapshot ioStatistics, + final Configuration destConf) { + // create a success data structure + SuccessData successData = new SuccessData(); + successData.setCommitter(getName()); + successData.setJobId(uuid); + successData.setJobIdSource(uuidSource.getText()); + successData.setDescription(getRole()); + successData.setHostname(NetUtils.getLocalHostname()); + Date now = new Date(); + successData.setTimestamp(now.getTime()); + successData.setDate(now.toString()); + if (filenames != null) { + successData.setFilenames(filenames); + } + successData.getIOStatistics().aggregate(ioStatistics); + // attach some config options as diagnostics to assist + // in debugging performance issues. + + // commit thread pool size + successData.addDiagnostic(FS_S3A_COMMITTER_THREADS, + Integer.toString(getJobCommitThreadCount(context))); + + // and filesystem http connection and thread pool sizes + if (destConf != null) { + successData.addDiagnostic(MAXIMUM_CONNECTIONS, + destConf.get(MAXIMUM_CONNECTIONS, + Integer.toString(DEFAULT_MAXIMUM_CONNECTIONS))); + successData.addDiagnostic(MAX_TOTAL_TASKS, + destConf.get(MAX_TOTAL_TASKS, + Integer.toString(DEFAULT_MAX_TOTAL_TASKS))); + } + return successData; } /** @@ -556,7 +614,11 @@ public void setupJob(JobContext context) throws IOException { @Override public void setupTask(TaskAttemptContext context) throws IOException { TaskAttemptID attemptID = context.getTaskAttemptID(); - updateCommonContext(); + + // update the context so that task IO in the same thread has + // the relevant values. + new AuditContextUpdater(context) + .updateCurrentAuditContext(); try (DurationInfo d = new DurationInfo(LOG, "Setup Task %s", attemptID)) { @@ -572,6 +634,9 @@ && getUUIDSource() == JobUUIDSource.GeneratedLocally) { } Path taskAttemptPath = getTaskAttemptPath(context); FileSystem fs = taskAttemptPath.getFileSystem(getConf()); + // delete that ta path if somehow it was there + fs.delete(taskAttemptPath, true); + // create an empty directory fs.mkdirs(taskAttemptPath); } } @@ -596,25 +661,23 @@ protected FileSystem getTaskAttemptFilesystem(TaskAttemptContext context) * On a failure or abort of a single file's commit, all its uploads are * aborted. * The revert operation lists the files already committed and deletes them. - * @param context job context + * @param commitContext commit context * @param pending pending uploads * @throws IOException on any failure */ protected void commitPendingUploads( - final JobContext context, + final CommitContext commitContext, final ActiveCommit pending) throws IOException { if (pending.isEmpty()) { LOG.warn("{}: No pending uploads to commit", getRole()); } try (DurationInfo ignored = new DurationInfo(LOG, - "committing the output of %s task(s)", pending.size()); - CommitOperations.CommitContext commitContext - = initiateCommitOperation()) { + "committing the output of %s task(s)", pending.size())) { - Tasks.foreach(pending.getSourceFiles()) + TaskPool.foreach(pending.getSourceFiles()) .stopOnFailure() .suppressExceptions(false) - .executeWith(buildSubmitter(context)) + .executeWith(commitContext.getOuterSubmitter()) .abortWith(status -> loadAndAbort(commitContext, pending, status, true, false)) .revertWith(status -> @@ -629,35 +692,39 @@ protected void commitPendingUploads( * This check avoids the situation where the inability to read * a file only surfaces partway through the job commit, so * results in the destination being tainted. - * @param context job context + * @param commitContext commit context * @param pending the pending operations * @throws IOException any failure */ protected void precommitCheckPendingFiles( - final JobContext context, + final CommitContext commitContext, final ActiveCommit pending) throws IOException { FileSystem sourceFS = pending.getSourceFS(); try (DurationInfo ignored = new DurationInfo(LOG, "Preflight Load of pending files")) { - Tasks.foreach(pending.getSourceFiles()) + TaskPool.foreach(pending.getSourceFiles()) .stopOnFailure() .suppressExceptions(false) - .executeWith(buildSubmitter(context)) - .run(status -> PendingSet.load(sourceFS, status)); + .executeWith(commitContext.getOuterSubmitter()) + .run(status -> PersistentCommitData.load(sourceFS, status, + commitContext.getPendingSetSerializer())); } } /** * Load a pendingset file and commit all of its contents. + * Invoked within a parallel run; the commitContext thread + * pool is already busy/possibly full, so do not + * execute work through the same submitter. * @param commitContext context to commit through * @param activeCommit commit state * @param status file to load * @throws IOException failure */ private void loadAndCommit( - final CommitOperations.CommitContext commitContext, + final CommitContext commitContext, final ActiveCommit activeCommit, final FileStatus status) throws IOException { @@ -665,18 +732,20 @@ private void loadAndCommit( try (DurationInfo ignored = new DurationInfo(LOG, "Loading and committing files in pendingset %s", path)) { - PendingSet pendingSet = PendingSet.load(activeCommit.getSourceFS(), - status); + PendingSet pendingSet = PersistentCommitData.load( + activeCommit.getSourceFS(), + status, + commitContext.getPendingSetSerializer()); String jobId = pendingSet.getJobId(); if (!StringUtils.isEmpty(jobId) && !getUUID().equals(jobId)) { throw new PathCommitException(path, String.format("Mismatch in Job ID (%s) and commit job ID (%s)", getUUID(), jobId)); } - Tasks.foreach(pendingSet.getCommits()) + TaskPool.foreach(pendingSet.getCommits()) .stopOnFailure() .suppressExceptions(false) - .executeWith(singleThreadSubmitter()) + .executeWith(commitContext.getInnerSubmitter()) .onFailure((commit, exception) -> commitContext.abortSingleCommit(commit)) .abortWith(commitContext::abortSingleCommit) @@ -692,22 +761,27 @@ private void loadAndCommit( /** * Load a pendingset file and revert all of its contents. + * Invoked within a parallel run; the commitContext thread + * pool is already busy/possibly full, so do not + * execute work through the same submitter. * @param commitContext context to commit through * @param activeCommit commit state * @param status status of file to load * @throws IOException failure */ private void loadAndRevert( - final CommitOperations.CommitContext commitContext, + final CommitContext commitContext, final ActiveCommit activeCommit, final FileStatus status) throws IOException { final Path path = status.getPath(); try (DurationInfo ignored = new DurationInfo(LOG, false, "Committing %s", path)) { - PendingSet pendingSet = PendingSet.load(activeCommit.getSourceFS(), - status); - Tasks.foreach(pendingSet.getCommits()) + PendingSet pendingSet = PersistentCommitData.load( + activeCommit.getSourceFS(), + status, + commitContext.getPendingSetSerializer()); + TaskPool.foreach(pendingSet.getCommits()) .suppressExceptions(true) .run(commitContext::revertCommit); } @@ -715,6 +789,9 @@ private void loadAndRevert( /** * Load a pendingset file and abort all of its contents. + * Invoked within a parallel run; the commitContext thread + * pool is already busy/possibly full, so do not + * execute work through the same submitter. * @param commitContext context to commit through * @param activeCommit commit state * @param status status of file to load @@ -722,7 +799,7 @@ private void loadAndRevert( * @throws IOException failure */ private void loadAndAbort( - final CommitOperations.CommitContext commitContext, + final CommitContext commitContext, final ActiveCommit activeCommit, final FileStatus status, final boolean suppressExceptions, @@ -731,11 +808,13 @@ private void loadAndAbort( final Path path = status.getPath(); try (DurationInfo ignored = new DurationInfo(LOG, false, "Aborting %s", path)) { - PendingSet pendingSet = PendingSet.load(activeCommit.getSourceFS(), - status); + PendingSet pendingSet = PersistentCommitData.load( + activeCommit.getSourceFS(), + status, + commitContext.getPendingSetSerializer()); FileSystem fs = getDestFS(); - Tasks.foreach(pendingSet.getCommits()) - .executeWith(singleThreadSubmitter()) + TaskPool.foreach(pendingSet.getCommits()) + .executeWith(commitContext.getInnerSubmitter()) .suppressExceptions(suppressExceptions) .run(commit -> { try { @@ -752,28 +831,51 @@ private void loadAndAbort( } /** - * Start the final commit/abort commit operations. + * Start the final job commit/abort commit operations. + * @param context job context * @return a commit context through which the operations can be invoked. * @throws IOException failure. */ - protected CommitOperations.CommitContext initiateCommitOperation() + protected CommitContext initiateJobOperation( + final JobContext context) throws IOException { - return getCommitOperations().initiateCommitOperation(getOutputPath()); + + return getCommitOperations().createCommitContext( + context, + getOutputPath(), + getJobCommitThreadCount(context)); + } + /** + * Start a ask commit/abort commit operations. + * This may have a different thread count. + * @param context job or task context + * @return a commit context through which the operations can be invoked. + * @throws IOException failure. + */ + protected CommitContext initiateTaskOperation( + final JobContext context) + throws IOException { + + return getCommitOperations().createCommitContext( + context, + getOutputPath(), + getTaskCommitThreadCount(context)); } /** * Internal Job commit operation: where the S3 requests are made * (potentially in parallel). - * @param context job context + * @param commitContext commit context * @param pending pending commits * @throws IOException any failure */ - protected void commitJobInternal(JobContext context, - ActiveCommit pending) + protected void commitJobInternal( + final CommitContext commitContext, + final ActiveCommit pending) throws IOException { trackDurationOfInvocation(committerStatistics, COMMITTER_COMMIT_JOB.getSymbol(), - () -> commitPendingUploads(context, pending)); + () -> commitPendingUploads(commitContext, pending)); } @Override @@ -782,7 +884,9 @@ public void abortJob(JobContext context, JobStatus.State state) LOG.info("{}: aborting job {} in state {}", getRole(), jobIdString(context), state); // final cleanup operations - abortJobInternal(context, false); + try (CommitContext commitContext = initiateJobOperation(context)){ + abortJobInternal(commitContext, false); + } } @@ -790,30 +894,33 @@ public void abortJob(JobContext context, JobStatus.State state) * The internal job abort operation; can be overridden in tests. * This must clean up operations; it is called when a commit fails, as * well as in an {@link #abortJob(JobContext, JobStatus.State)} call. - * The base implementation calls {@link #cleanup(JobContext, boolean)} + * The base implementation calls {@link #cleanup(CommitContext, boolean)} * so cleans up the filesystems and destroys the thread pool. * Subclasses must always invoke this superclass method after their * own operations. - * @param context job context + * Creates and closes its own commit context. + * + * @param commitContext commit context * @param suppressExceptions should exceptions be suppressed? * @throws IOException any IO problem raised when suppressExceptions is false. */ - protected void abortJobInternal(JobContext context, + protected void abortJobInternal(CommitContext commitContext, boolean suppressExceptions) throws IOException { - cleanup(context, suppressExceptions); + cleanup(commitContext, suppressExceptions); } /** * Abort all pending uploads to the destination directory during * job cleanup operations. * Note: this instantiates the thread pool if required -so - * {@link #destroyThreadPool()} must be called after this. * @param suppressExceptions should exceptions be suppressed + * @param commitContext commit context * @throws IOException IO problem */ protected void abortPendingUploadsInCleanup( - boolean suppressExceptions) throws IOException { + boolean suppressExceptions, + CommitContext commitContext) throws IOException { // return early if aborting is disabled. if (!shouldAbortUploadsInCleanup()) { LOG.debug("Not cleanup up pending uploads to {} as {} is false ", @@ -824,9 +931,7 @@ protected void abortPendingUploadsInCleanup( Path dest = getOutputPath(); try (DurationInfo ignored = new DurationInfo(LOG, "Aborting all pending commits under %s", - dest); - CommitOperations.CommitContext commitContext - = initiateCommitOperation()) { + dest)) { CommitOperations ops = getCommitOperations(); List pending; try { @@ -840,8 +945,8 @@ protected void abortPendingUploadsInCleanup( LOG.warn("{} pending uploads were found -aborting", pending.size()); LOG.warn("If other tasks/jobs are writing to {}," + "this action may cause them to fail", dest); - Tasks.foreach(pending) - .executeWith(buildSubmitter(getJobContext())) + TaskPool.foreach(pending) + .executeWith(commitContext.getOuterSubmitter()) .suppressExceptions(suppressExceptions) .run(u -> commitContext.abortMultipartCommit( u.getKey(), u.getUploadId())); @@ -863,12 +968,12 @@ private boolean shouldAbortUploadsInCleanup() { * they can be loaded. * The Magic committer does not, because of the overhead of reading files * from S3 makes it too expensive. - * @param context job context + * @param commitContext commit context * @param pending the pending operations * @throws IOException any failure */ @VisibleForTesting - public void preCommitJob(JobContext context, + public void preCommitJob(CommitContext commitContext, ActiveCommit pending) throws IOException { } @@ -890,23 +995,55 @@ public void preCommitJob(JobContext context, @Override public void commitJob(JobContext context) throws IOException { String id = jobIdString(context); + // the commit context is created outside a try-with-resources block + // so it can be used in exception handling. + CommitContext commitContext = null; + + SuccessData successData = null; + IOException failure = null; + String stage = "preparing"; try (DurationInfo d = new DurationInfo(LOG, "%s: commitJob(%s)", getRole(), id)) { + commitContext = initiateJobOperation(context); ActiveCommit pending - = listPendingUploadsToCommit(context); - preCommitJob(context, pending); - commitJobInternal(context, pending); + = listPendingUploadsToCommit(commitContext); + stage = "precommit"; + preCommitJob(commitContext, pending); + stage = "commit"; + commitJobInternal(commitContext, pending); + stage = "completed"; jobCompleted(true); - maybeCreateSuccessMarkerFromCommits(context, pending); - cleanup(context, false); + stage = "marker"; + successData = maybeCreateSuccessMarkerFromCommits(context, pending); + stage = "cleanup"; + cleanup(commitContext, false); } catch (IOException e) { + // failure. record it for the summary + failure = e; LOG.warn("Commit failure for job {}", id, e); jobCompleted(false); - abortJobInternal(context, true); + abortJobInternal(commitContext, true); throw e; } finally { - resetCommonContext(); + // save the report summary, even on failure + if (commitContext != null) { + if (successData == null) { + // if the commit did not get as far as creating success data, create one. + successData = createSuccessData(context, null, null, + getDestFS().getConf()); + } + // save quietly, so no exceptions are raised + maybeSaveSummary(stage, + commitContext, + successData, + failure, + true, + true); + // and close that commit context + commitContext.close(); + } } + } /** @@ -925,30 +1062,28 @@ protected void jobCompleted(boolean success) { /** * Get the list of pending uploads for this job attempt. - * @param context job context + * @param commitContext commit context * @return a list of pending uploads. * @throws IOException Any IO failure */ protected abstract ActiveCommit listPendingUploadsToCommit( - JobContext context) + CommitContext commitContext) throws IOException; /** * Cleanup the job context, including aborting anything pending * and destroying the thread pool. - * @param context job context + * @param commitContext commit context * @param suppressExceptions should exceptions be suppressed? * @throws IOException any failure if exceptions were not suppressed. */ - protected void cleanup(JobContext context, + protected void cleanup(CommitContext commitContext, boolean suppressExceptions) throws IOException { try (DurationInfo d = new DurationInfo(LOG, - "Cleanup job %s", jobIdString(context))) { - abortPendingUploadsInCleanup(suppressExceptions); + "Cleanup job %s", jobIdString(commitContext.getJobContext()))) { + abortPendingUploadsInCleanup(suppressExceptions, commitContext); } finally { - destroyThreadPool(); cleanupStagingDirs(); - resetCommonContext(); } } @@ -958,8 +1093,9 @@ public void cleanupJob(JobContext context) throws IOException { String r = getRole(); String id = jobIdString(context); LOG.warn("{}: using deprecated cleanupJob call for {}", r, id); - try (DurationInfo d = new DurationInfo(LOG, "%s: cleanup Job %s", r, id)) { - cleanup(context, true); + try (DurationInfo d = new DurationInfo(LOG, "%s: cleanup Job %s", r, id); + CommitContext commitContext = initiateJobOperation(context)) { + cleanup(commitContext, true); } } @@ -1016,137 +1152,26 @@ protected String getRole() { return role; } - /** - * Returns an {@link Tasks.Submitter} for parallel tasks. The number of - * threads in the thread-pool is set by fs.s3a.committer.threads. - * If num-threads is 0, this will return null; - * this is used in Tasks as a cue - * to switch to single-threaded execution. - * - * @param context the JobContext for this commit - * @return a submitter or null - */ - protected Tasks.Submitter buildSubmitter( - JobContext context) { - if (getThreadCount(context) > 0) { - return new PoolSubmitter(context); - } else { - return null; - } - } - - /** - * Returns an {@link ExecutorService} for parallel tasks. The number of - * threads in the thread-pool is set by fs.s3a.committer.threads. - * If num-threads is 0, this will raise an exception. - * - * @param context the JobContext for this commit - * @param numThreads threads - * @return an {@link ExecutorService} for the number of threads - */ - private synchronized ExecutorService buildThreadPool( - JobContext context, int numThreads) { - Preconditions.checkArgument(numThreads > 0, - "Cannot create a thread pool with no threads"); - if (threadPool == null) { - LOG.debug("{}: creating thread pool of size {}", getRole(), numThreads); - threadPool = HadoopExecutors.newFixedThreadPool(numThreads, - new ThreadFactoryBuilder() - .setDaemon(true) - .setNameFormat(THREAD_PREFIX + context.getJobID() + "-%d") - .build()); - } - return threadPool; - } - /** * Get the thread count for this job's commit operations. * @param context the JobContext for this commit * @return a possibly zero thread count. */ - private int getThreadCount(final JobContext context) { + private int getJobCommitThreadCount(final JobContext context) { return context.getConfiguration().getInt( FS_S3A_COMMITTER_THREADS, DEFAULT_COMMITTER_THREADS); } /** - * Submit a runnable. - * This will demand-create the thread pool if needed. - *

- * This is synchronized to ensure the thread pool is always valid when - * work is synchronized. See HADOOP-16798. + * Get the thread count for this task's commit operations. * @param context the JobContext for this commit - * @param task task to execute - * @return the future of the submitted task. + * @return a possibly zero thread count. */ - private synchronized Future submitRunnable( - final JobContext context, - final Runnable task) { - return buildThreadPool(context, getThreadCount(context)).submit(task); - } - - /** - * The real task submitter, which hands off the work to - * the current thread pool. - */ - private final class PoolSubmitter implements Tasks.Submitter { - - private final JobContext context; - - private final int numThreads; - - private PoolSubmitter(final JobContext context) { - this.numThreads = getThreadCount(context); - Preconditions.checkArgument(numThreads > 0, - "Cannot create a thread pool with no threads"); - this.context = context; - } - - @Override - public Future submit(final Runnable task) { - return submitRunnable(context, task); - } - - } - - /** - * Destroy any thread pools; wait for that to finish, - * but don't overreact if it doesn't finish in time. - */ - protected void destroyThreadPool() { - ExecutorService pool; - // reset the thread pool in a sync block, then shut it down - // afterwards. This allows for other threads to create a - // new thread pool on demand. - synchronized(this) { - pool = this.threadPool; - threadPool = null; - } - if (pool != null) { - LOG.debug("Destroying thread pool"); - HadoopExecutors.shutdown(pool, LOG, - THREAD_POOL_SHUTDOWN_DELAY_SECONDS, TimeUnit.SECONDS); - } - } - - /** - * Get the thread pool for executing the single file commit/revert - * within the commit of all uploads of a single task. - * This is currently null; it is here to allow the Tasks class to - * provide the logic for execute/revert. - * @return null. always. - */ - protected final synchronized Tasks.Submitter singleThreadSubmitter() { - return null; - } - - /** - * Does this committer have a thread pool? - * @return true if a thread pool exists. - */ - public synchronized boolean hasThreadPool() { - return threadPool != null; + private int getTaskCommitThreadCount(final JobContext context) { + return context.getConfiguration().getInt( + FS_S3A_COMMITTER_THREADS, + DEFAULT_COMMITTER_THREADS); } /** @@ -1164,24 +1189,23 @@ protected void deleteTaskAttemptPathQuietly(TaskAttemptContext context) { * Abort all pending uploads in the list. * This operation is used by the magic committer as part of its * rollback after a failure during task commit. - * @param context job context + * @param commitContext commit context * @param pending pending uploads * @param suppressExceptions should exceptions be suppressed * @throws IOException any exception raised */ - protected void abortPendingUploads(JobContext context, - List pending, - boolean suppressExceptions) + protected void abortPendingUploads( + final CommitContext commitContext, + final List pending, + final boolean suppressExceptions) throws IOException { if (pending == null || pending.isEmpty()) { LOG.info("{}: no pending commits to abort", getRole()); } else { try (DurationInfo d = new DurationInfo(LOG, - "Aborting %s uploads", pending.size()); - CommitOperations.CommitContext commitContext - = initiateCommitOperation()) { - Tasks.foreach(pending) - .executeWith(buildSubmitter(context)) + "Aborting %s uploads", pending.size())) { + TaskPool.foreach(pending) + .executeWith(commitContext.getOuterSubmitter()) .suppressExceptions(suppressExceptions) .run(commitContext::abortSingleCommit); } @@ -1190,27 +1214,24 @@ protected void abortPendingUploads(JobContext context, /** * Abort all pending uploads in the list. - * @param context job context + * @param commitContext commit context * @param pending pending uploads * @param suppressExceptions should exceptions be suppressed? * @param deleteRemoteFiles should remote files be deleted? * @throws IOException any exception raised */ protected void abortPendingUploads( - final JobContext context, + final CommitContext commitContext, final ActiveCommit pending, final boolean suppressExceptions, final boolean deleteRemoteFiles) throws IOException { - if (pending.isEmpty()) { LOG.info("{}: no pending commits to abort", getRole()); } else { try (DurationInfo d = new DurationInfo(LOG, - "Aborting %s uploads", pending.size()); - CommitOperations.CommitContext commitContext - = initiateCommitOperation()) { - Tasks.foreach(pending.getSourceFiles()) - .executeWith(buildSubmitter(context)) + "Aborting %s uploads", pending.size())) { + TaskPool.foreach(pending.getSourceFiles()) + .executeWith(commitContext.getOuterSubmitter()) .suppressExceptions(suppressExceptions) .run(path -> loadAndAbort(commitContext, @@ -1392,13 +1413,7 @@ public String toString() { */ protected final void updateCommonContext() { currentAuditContext().put(AuditConstants.PARAM_JOB_ID, uuid); - } - /** - * Remove JobID from the current thread's context. - */ - protected final void resetCommonContext() { - currentAuditContext().remove(AuditConstants.PARAM_JOB_ID); } protected AuditSpanSource getAuditSpanSource() { @@ -1424,6 +1439,77 @@ protected AuditSpan startOperation(String name, return getAuditSpanSource().createSpan(name, path1, path2); } + + /** + * Save a summary to the report dir if the config option + * is set. + * The report will be updated with the current active stage, + * and if {@code thrown} is non-null, it will be added to the + * diagnostics (and the job tagged as a failure). + * Static for testability. + * @param activeStage active stage + * @param context commit context. + * @param report summary file. + * @param thrown any exception indicting failure. + * @param quiet should exceptions be swallowed. + * @param overwrite should the existing file be overwritten + * @return the path of a file, if successfully saved + * @throws IOException if a failure occured and quiet==false + */ + private static Path maybeSaveSummary( + String activeStage, + CommitContext context, + SuccessData report, + Throwable thrown, + boolean quiet, + boolean overwrite) throws IOException { + Configuration conf = context.getConf(); + String reportDir = conf.getTrimmed(OPT_SUMMARY_REPORT_DIR, ""); + if (reportDir.isEmpty()) { + LOG.debug("No summary directory set in " + OPT_SUMMARY_REPORT_DIR); + return null; + } + LOG.debug("Summary directory set to {}", reportDir); + + Path reportDirPath = new Path(reportDir); + Path path = new Path(reportDirPath, + createJobSummaryFilename(context.getJobId())); + + if (thrown != null) { + report.recordJobFailure(thrown); + } + report.putDiagnostic(STAGE, activeStage); + // the store operations here is explicitly created for the FS where + // the reports go, which may not be the target FS of the job. + + final FileSystem fs = path.getFileSystem(conf); + try (ManifestStoreOperations operations = new ManifestStoreOperationsThroughFileSystem( + fs)) { + if (!overwrite) { + // check for file existence so there is no need to worry about + // precisely what exception is raised when overwrite=false and dest file + // exists + try { + FileStatus st = operations.getFileStatus(path); + // get here and the file exists + LOG.debug("Report already exists: {}", st); + return null; + } catch (FileNotFoundException ignored) { + } + } + report.save(fs, path, SuccessData.serializer()); + LOG.info("Job summary saved to {}", path); + return path; + } catch (IOException e) { + LOG.debug("Failed to save summary to {}", path, e); + if (quiet) { + return null; + } else { + throw e; + } + } + } + /** * State of the active commit operation. * @@ -1445,7 +1531,7 @@ protected AuditSpan startOperation(String name, * * */ - public static class ActiveCommit { + public static final class ActiveCommit { private static final AbstractS3ACommitter.ActiveCommit EMPTY = new ActiveCommit(null, new ArrayList<>()); @@ -1486,6 +1572,7 @@ public static class ActiveCommit { * @param sourceFS filesystem containing the list of pending files * @param sourceFiles .pendingset files to load and commit. */ + @SuppressWarnings("unchecked") public ActiveCommit( final FileSystem sourceFS, final List sourceFiles) { @@ -1496,13 +1583,14 @@ public ActiveCommit( /** * Create an active commit of the given pending files. * @param pendingFS source filesystem. - * @param statuses list of file status or subclass to use. + * @param statuses iterator of file status or subclass to use. * @return the commit + * @throws IOException if the iterator raises one. */ - public static ActiveCommit fromStatusList( + public static ActiveCommit fromStatusIterator( final FileSystem pendingFS, - final List statuses) { - return new ActiveCommit(pendingFS, statuses); + final RemoteIterator statuses) throws IOException { + return new ActiveCommit(pendingFS, toList(statuses)); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitConstants.java index bbc59f168f..b85ce27650 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitConstants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitConstants.java @@ -22,7 +22,6 @@ import org.apache.hadoop.classification.InterfaceStability; import static org.apache.hadoop.fs.s3a.Constants.XA_HEADER_PREFIX; -import static org.apache.hadoop.mapreduce.lib.output.PathOutputCommitterFactory.COMMITTER_FACTORY_SCHEME_PATTERN; /** * Constants for working with committers. @@ -57,6 +56,12 @@ private CommitConstants() { */ public static final String PENDINGSET_SUFFIX = ".pendingset"; + + /** + * Prefix to use for config options: {@value}. + */ + public static final String OPT_PREFIX = "fs.s3a.committer."; + /** * Flag to indicate whether support for the Magic committer is enabled * in the filesystem. @@ -121,9 +126,8 @@ private CommitConstants() { /** * Temp data which is not auto-committed: {@value}. - * Uses a different name from normal just to make clear it is different. */ - public static final String TEMP_DATA = "__temp-data"; + public static final String TEMP_DATA = TEMPORARY; /** @@ -144,7 +148,7 @@ private CommitConstants() { * Key to set for the S3A schema to use the specific committer. */ public static final String S3A_COMMITTER_FACTORY_KEY = String.format( - COMMITTER_FACTORY_SCHEME_PATTERN, "s3a"); + "mapreduce.outputcommitter.factory.scheme.s3a"); /** * S3 Committer factory: {@value}. @@ -222,13 +226,19 @@ private CommitConstants() { /** * Number of threads in committers for parallel operations on files * (upload, commit, abort, delete...): {@value}. + * Two thread pools this size are created, one for the outer + * task-level parallelism, and one for parallel execution + * within tasks (POSTs to commit individual uploads) + * If the value is negative, it is inverted and then multiplied + * by the number of cores in the CPU. */ public static final String FS_S3A_COMMITTER_THREADS = "fs.s3a.committer.threads"; + /** * Default value for {@link #FS_S3A_COMMITTER_THREADS}: {@value}. */ - public static final int DEFAULT_COMMITTER_THREADS = 8; + public static final int DEFAULT_COMMITTER_THREADS = 32; /** * Path in the cluster filesystem for temporary data: {@value}. @@ -330,4 +340,18 @@ private CommitConstants() { public static final String XA_MAGIC_MARKER = XA_HEADER_PREFIX + X_HEADER_MAGIC_MARKER; + /** + * Task Attempt ID query header: {@value}. + */ + public static final String PARAM_TASK_ATTEMPT_ID = "ta"; + + /** + * Directory for saving job summary reports. + * These are the _SUCCESS files, but are saved even on + * job failures. + * Value: {@value}. + */ + public static final String OPT_SUMMARY_REPORT_DIR = + OPT_PREFIX + "summary.report.directory"; + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitterStatisticNames.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitterStatisticNames.java new file mode 100644 index 0000000000..20af292b3e --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitterStatisticNames.java @@ -0,0 +1,211 @@ +/* + * 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.s3a.commit; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.statistics.StoreStatisticNames; + +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_RENAME; + +/** + * Statistic names for committers. + * Please keep in sync with org.apache.hadoop.fs.s3a.Statistic + * so that S3A and manifest committers are in sync. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public final class CommitterStatisticNames { + + + /** Amount of data committed: {@value}. */ + public static final String COMMITTER_BYTES_COMMITTED_COUNT = + "committer_bytes_committed"; + + /** Duration Tracking of time to commit an entire job: {@value}. */ + public static final String COMMITTER_COMMIT_JOB = + "committer_commit_job"; + + /** Number of files committed: {@value}. */ + public static final String COMMITTER_FILES_COMMITTED_COUNT = + "committer_files_committed"; + /** "Count of successful tasks:: {@value}. */ + public static final String COMMITTER_TASKS_COMPLETED_COUNT = + "committer_tasks_completed"; + + /** Count of failed tasks: {@value}. */ + public static final String COMMITTER_TASKS_FAILED_COUNT = + "committer_tasks_failed"; + + /** Count of commits aborted: {@value}. */ + public static final String COMMITTER_COMMITS_ABORTED_COUNT = + "committer_commits_aborted"; + + /** Count of commits reverted: {@value}. */ + public static final String COMMITTER_COMMITS_REVERTED_COUNT = + "committer_commits_reverted"; + + /** Count of commits failed: {@value}. */ + public static final String COMMITTER_COMMITS_FAILED = + "committer_commits" + StoreStatisticNames.SUFFIX_FAILURES; + + /** + * The number of files in a task. This will be a MeanStatistic. + */ + public static final String COMMITTER_FILE_COUNT_MEAN = + "committer_task_file_count"; + + /** + * File Size. + */ + public static final String COMMITTER_FILE_SIZE_MEAN = + "committer_task_file_size"; + + /** + * What is a task attempt's directory count. + */ + public static final String COMMITTER_TASK_DIRECTORY_COUNT_MEAN = + "committer_task_directory_count"; + + /** + * What is the depth of a task attempt's directory tree. + */ + public static final String COMMITTER_TASK_DIRECTORY_DEPTH_MEAN = + "committer_task_directory_depth"; + + /** + * The number of files in a task. This will be a MeanStatistic. + */ + public static final String COMMITTER_TASK_FILE_COUNT_MEAN = + "committer_task_file_count"; + + /** + * The number of files in a task. This will be a MeanStatistic. + */ + public static final String COMMITTER_TASK_FILE_SIZE_MEAN = + "committer_task_file_size"; + + /** Directory creation {@value}. */ + public static final String OP_CREATE_DIRECTORIES = "op_create_directories"; + + /** Creating a single directory {@value}. */ + public static final String OP_CREATE_ONE_DIRECTORY = + "op_create_one_directory"; + + /** Directory scan {@value}. */ + public static final String OP_DIRECTORY_SCAN = "op_directory_scan"; + + /** + * Overall job commit {@value}. + */ + public static final String OP_STAGE_JOB_COMMIT = COMMITTER_COMMIT_JOB; + + /** {@value}. */ + public static final String OP_LOAD_ALL_MANIFESTS = "op_load_all_manifests"; + + /** + * Load a task manifest: {@value}. + */ + public static final String OP_LOAD_MANIFEST = "op_load_manifest"; + + /** Rename a file: {@value}. */ + public static final String OP_RENAME_FILE = OP_RENAME; + + /** + * Save a task manifest: {@value}. + */ + public static final String OP_SAVE_TASK_MANIFEST = + "op_task_stage_save_task_manifest"; + + /** + * Task abort: {@value}. + */ + public static final String OP_STAGE_TASK_ABORT_TASK + = "op_task_stage_abort_task"; + + /** + * Job abort: {@value}. + */ + public static final String OP_STAGE_JOB_ABORT = "op_job_stage_abort"; + + /** + * Job cleanup: {@value}. + */ + public static final String OP_STAGE_JOB_CLEANUP = "op_job_stage_cleanup"; + + /** + * Prepare Directories Stage: {@value}. + */ + public static final String OP_STAGE_JOB_CREATE_TARGET_DIRS = + "op_job_stage_create_target_dirs"; + + /** + * Load Manifest Stage: {@value}. + */ + public static final String OP_STAGE_JOB_LOAD_MANIFESTS = + "op_job_stage_load_manifests"; + + /** + * Rename files stage duration: {@value}. + */ + public static final String OP_STAGE_JOB_RENAME_FILES = + "op_job_stage_rename_files"; + + + /** + * Job Setup Stage: {@value}. + */ + public static final String OP_STAGE_JOB_SETUP = "op_job_stage_setup"; + + /** + * Job saving _SUCCESS marker Stage: {@value}. + */ + public static final String OP_STAGE_JOB_SAVE_SUCCESS = + "op_job_stage_save_success_marker"; + + /** + * Output Validation (within job commit) Stage: {@value}. + */ + public static final String OP_STAGE_JOB_VALIDATE_OUTPUT = + + "op_job_stage_optional_validate_output"; + /** + * Task saving manifest file Stage: {@value}. + */ + public static final String OP_STAGE_TASK_SAVE_MANIFEST = + "op_task_stage_save_manifest"; + + /** + * Task Setup Stage: {@value}. + */ + public static final String OP_STAGE_TASK_SETUP = "op_task_stage_setup"; + + + /** + * Task Commit Stage: {@value}. + */ + public static final String OP_STAGE_TASK_COMMIT = "op_stage_task_commit"; + + /** Task Scan directory Stage: {@value}. */ + public static final String OP_STAGE_TASK_SCAN_DIRECTORY + = "op_stage_task_scan_directory"; + + private CommitterStatisticNames() { + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/InternalCommitterConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/InternalCommitterConstants.java index fcafdd1ed1..b2d4bfaeea 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/InternalCommitterConstants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/InternalCommitterConstants.java @@ -35,6 +35,12 @@ @InterfaceStability.Unstable public final class InternalCommitterConstants { + /** + * How long threads in the thread pool stay alive when + * idle. Value in seconds: {@value}. + */ + public static final long THREAD_KEEP_ALIVE_TIME = 60L; + private InternalCommitterConstants() { } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java index 41d36b2a8d..e6524c9196 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java @@ -28,6 +28,7 @@ import org.apache.hadoop.fs.s3a.Statistic; import org.apache.hadoop.fs.s3a.commit.magic.MagicCommitTracker; import org.apache.hadoop.fs.s3a.impl.AbstractStoreOperation; +import org.apache.hadoop.fs.s3a.statistics.PutTrackerStatistics; import static org.apache.hadoop.fs.s3a.commit.MagicCommitPaths.*; @@ -38,8 +39,9 @@ * in this case: *
    *
  1. {@link #isMagicCommitPath(Path)} will always return false.
  2. - *
  3. {@link #createTracker(Path, String)} will always return an instance - * of {@link PutTracker}.
  4. + *
  5. {@link #isUnderMagicPath(Path)} will always return false.
  6. + *
  7. {@link #createTracker(Path, String, PutTrackerStatistics)} will always + * return an instance of {@link PutTracker}.
  8. *
* *

Important

: must not directly or indirectly import a class which @@ -88,9 +90,11 @@ public String keyOfFinalDestination(List elements, String key) { * for the commit tracker. * @param path path of nominal write * @param key key of path of nominal write + * @param trackerStatistics tracker statistics * @return the tracker for this operation. */ - public PutTracker createTracker(Path path, String key) { + public PutTracker createTracker(Path path, String key, + PutTrackerStatistics trackerStatistics) { final List elements = splitPathToElements(path); PutTracker tracker; @@ -106,7 +110,8 @@ public PutTracker createTracker(Path path, String key) { key, destKey, pendingsetPath, - owner.getWriteOperationHelper()); + owner.getWriteOperationHelper(), + trackerStatistics); LOG.debug("Created {}", tracker); } else { LOG.warn("File being created has a \"magic\" path, but the filesystem" @@ -184,4 +189,13 @@ private boolean isCommitMetadataFile(List elements) { || last.endsWith(CommitConstants.PENDINGSET_SUFFIX); } + /** + * Is this path in/under a magic path...regardless of file type. + * This is used to optimize create() operations. + * @param path path to check + * @return true if the path is one a magic file write expects. + */ + public boolean isUnderMagicPath(Path path) { + return magicCommitEnabled && isMagicPath(splitPathToElements(path)); + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/Tasks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/Tasks.java deleted file mode 100644 index c318e86605..0000000000 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/Tasks.java +++ /dev/null @@ -1,423 +0,0 @@ -/* - * 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.s3a.commit; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Iterator; -import java.util.List; -import java.util.Queue; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Utility class for parallel execution, takes closures for the various - * actions. - * There is no retry logic: it is expected to be handled by the closures. - */ -public final class Tasks { - private static final Logger LOG = LoggerFactory.getLogger(Tasks.class); - - private Tasks() { - } - - /** - * Callback invoked to process an item. - * @param item type being processed - * @param exception class which may be raised - */ - @FunctionalInterface - public interface Task { - void run(I item) throws E; - } - - /** - * Callback invoked on a failure. - * @param item type being processed - * @param exception class which may be raised - */ - @FunctionalInterface - public interface FailureTask { - - /** - * process a failure. - * @param item item the task is processing - * @param exception the exception which was raised. - * @throws E Exception of type E - */ - void run(I item, Exception exception) throws E; - } - - /** - * Builder for task execution. - * @param item type - */ - public static class Builder { - private final Iterable items; - private Submitter service = null; - private FailureTask onFailure = null; - private boolean stopOnFailure = false; - private boolean suppressExceptions = false; - private Task revertTask = null; - private boolean stopRevertsOnFailure = false; - private Task abortTask = null; - private boolean stopAbortsOnFailure = false; - - /** - * Create the builder. - * @param items items to process - */ - Builder(Iterable items) { - this.items = items; - } - - /** - * Declare executor service: if null, the tasks are executed in a single - * thread. - * @param submitter service to schedule tasks with. - * @return this builder. - */ - public Builder executeWith(Submitter submitter) { - this.service = submitter; - return this; - } - - public Builder onFailure(FailureTask task) { - this.onFailure = task; - return this; - } - - public Builder stopOnFailure() { - this.stopOnFailure = true; - return this; - } - - public Builder suppressExceptions() { - return suppressExceptions(true); - } - - public Builder suppressExceptions(boolean suppress) { - this.suppressExceptions = suppress; - return this; - } - - public Builder revertWith(Task task) { - this.revertTask = task; - return this; - } - - public Builder stopRevertsOnFailure() { - this.stopRevertsOnFailure = true; - return this; - } - - public Builder abortWith(Task task) { - this.abortTask = task; - return this; - } - - public Builder stopAbortsOnFailure() { - this.stopAbortsOnFailure = true; - return this; - } - - public boolean run(Task task) throws E { - if (service != null) { - return runParallel(task); - } else { - return runSingleThreaded(task); - } - } - - private boolean runSingleThreaded(Task task) - throws E { - List succeeded = new ArrayList<>(); - List exceptions = new ArrayList<>(); - - Iterator iterator = items.iterator(); - boolean threw = true; - try { - while (iterator.hasNext()) { - I item = iterator.next(); - try { - task.run(item); - succeeded.add(item); - - } catch (Exception e) { - exceptions.add(e); - - if (onFailure != null) { - try { - onFailure.run(item, e); - } catch (Exception failException) { - LOG.error("Failed to clean up on failure", e); - // keep going - } - } - - if (stopOnFailure) { - break; - } - } - } - - threw = false; - - } finally { - // threw handles exceptions that were *not* caught by the catch block, - // and exceptions that were caught and possibly handled by onFailure - // are kept in exceptions. - if (threw || !exceptions.isEmpty()) { - if (revertTask != null) { - boolean failed = false; - for (I item : succeeded) { - try { - revertTask.run(item); - } catch (Exception e) { - LOG.error("Failed to revert task", e); - failed = true; - // keep going - } - if (stopRevertsOnFailure && failed) { - break; - } - } - } - - if (abortTask != null) { - boolean failed = false; - while (iterator.hasNext()) { - try { - abortTask.run(iterator.next()); - } catch (Exception e) { - failed = true; - LOG.error("Failed to abort task", e); - // keep going - } - if (stopAbortsOnFailure && failed) { - break; - } - } - } - } - } - - if (!suppressExceptions && !exceptions.isEmpty()) { - Tasks.throwOne(exceptions); - } - - return !threw && exceptions.isEmpty(); - } - - private boolean runParallel(final Task task) - throws E { - final Queue succeeded = new ConcurrentLinkedQueue<>(); - final Queue exceptions = new ConcurrentLinkedQueue<>(); - final AtomicBoolean taskFailed = new AtomicBoolean(false); - final AtomicBoolean abortFailed = new AtomicBoolean(false); - final AtomicBoolean revertFailed = new AtomicBoolean(false); - - List> futures = new ArrayList<>(); - - for (final I item : items) { - // submit a task for each item that will either run or abort the task - futures.add(service.submit(new Runnable() { - @Override - public void run() { - if (!(stopOnFailure && taskFailed.get())) { - // run the task - boolean threw = true; - try { - LOG.debug("Executing task"); - task.run(item); - succeeded.add(item); - LOG.debug("Task succeeded"); - - threw = false; - - } catch (Exception e) { - taskFailed.set(true); - exceptions.add(e); - LOG.info("Task failed", e); - - if (onFailure != null) { - try { - onFailure.run(item, e); - } catch (Exception failException) { - LOG.error("Failed to clean up on failure", e); - // swallow the exception - } - } - } finally { - if (threw) { - taskFailed.set(true); - } - } - - } else if (abortTask != null) { - // abort the task instead of running it - if (stopAbortsOnFailure && abortFailed.get()) { - return; - } - - boolean failed = true; - try { - LOG.info("Aborting task"); - abortTask.run(item); - failed = false; - } catch (Exception e) { - LOG.error("Failed to abort task", e); - // swallow the exception - } finally { - if (failed) { - abortFailed.set(true); - } - } - } - } - })); - } - - // let the above tasks complete (or abort) - waitFor(futures); - int futureCount = futures.size(); - futures.clear(); - - if (taskFailed.get() && revertTask != null) { - // at least one task failed, revert any that succeeded - LOG.info("Reverting all {} succeeded tasks from {} futures", - succeeded.size(), futureCount); - for (final I item : succeeded) { - futures.add(service.submit(() -> { - if (stopRevertsOnFailure && revertFailed.get()) { - return; - } - - boolean failed = true; - try { - revertTask.run(item); - failed = false; - } catch (Exception e) { - LOG.error("Failed to revert task", e); - // swallow the exception - } finally { - if (failed) { - revertFailed.set(true); - } - } - })); - } - - // let the revert tasks complete - waitFor(futures); - } - - if (!suppressExceptions && !exceptions.isEmpty()) { - Tasks.throwOne(exceptions); - } - - return !taskFailed.get(); - } - } - - /** - * Wait for all the futures to complete; there's a small sleep between - * each iteration; enough to yield the CPU. - * @param futures futures. - */ - private static void waitFor(Collection> futures) { - int size = futures.size(); - LOG.debug("Waiting for {} tasks to complete", size); - int oldNumFinished = 0; - while (true) { - int numFinished = (int) futures.stream().filter(Future::isDone).count(); - - if (oldNumFinished != numFinished) { - LOG.debug("Finished count -> {}/{}", numFinished, size); - oldNumFinished = numFinished; - } - - if (numFinished == size) { - // all of the futures are done, stop looping - break; - } else { - try { - Thread.sleep(10); - } catch (InterruptedException e) { - futures.forEach(future -> future.cancel(true)); - Thread.currentThread().interrupt(); - break; - } - } - } - } - - public static Builder foreach(Iterable items) { - return new Builder<>(items); - } - - public static Builder foreach(I[] items) { - return new Builder<>(Arrays.asList(items)); - } - - @SuppressWarnings("unchecked") - private static void throwOne( - Collection exceptions) - throws E { - Iterator iter = exceptions.iterator(); - Exception e = iter.next(); - Class exceptionClass = e.getClass(); - - while (iter.hasNext()) { - Exception other = iter.next(); - if (!exceptionClass.isInstance(other)) { - e.addSuppressed(other); - } - } - - Tasks.castAndThrow(e); - } - - @SuppressWarnings("unchecked") - private static void castAndThrow(Exception e) throws E { - if (e instanceof RuntimeException) { - throw (RuntimeException) e; - } - throw (E) e; - } - - /** - * Interface to whatever lets us submit tasks. - */ - public interface Submitter { - - /** - * Submit work. - * @param task task to execute - * @return the future of the submitted task. - */ - Future submit(Runnable task); - } - -} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java index 318896e236..108e6d45bc 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java @@ -38,6 +38,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.commit.ValidationFailure; +import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.util.JsonSerialization; @@ -63,8 +64,7 @@ @SuppressWarnings("unused") @InterfaceAudience.Private @InterfaceStability.Unstable -public class PendingSet extends PersistentCommitData - implements IOStatisticsSource { +public class PendingSet extends PersistentCommitData { private static final Logger LOG = LoggerFactory.getLogger(PendingSet.class); /** @@ -112,38 +112,13 @@ public PendingSet(int size) { } /** - * Get a JSON serializer for this class. + * Get a shared JSON serializer for this class. * @return a serializer. */ public static JsonSerialization serializer() { - return new JsonSerialization<>(PendingSet.class, false, true); + return new JsonSerialization<>(PendingSet.class, false, false); } - /** - * Load an instance from a file, then validate it. - * @param fs filesystem - * @param path path - * @return the loaded instance - * @throws IOException IO failure - * @throws ValidationFailure if the data is invalid - */ - public static PendingSet load(FileSystem fs, Path path) - throws IOException { - return load(fs, path, null); - } - - /** - * Load an instance from a file, then validate it. - * @param fs filesystem - * @param status status of file to load - * @return the loaded instance - * @throws IOException IO failure - * @throws ValidationFailure if the data is invalid - */ - public static PendingSet load(FileSystem fs, FileStatus status) - throws IOException { - return load(fs, status.getPath(), status); - } /** * Load an instance from a file, then validate it. @@ -211,8 +186,8 @@ public void validate() throws ValidationFailure { } @Override - public byte[] toBytes() throws IOException { - return serializer().toBytes(this); + public byte[] toBytes(JsonSerialization serializer) throws IOException { + return serializer.toBytes(this); } /** @@ -224,9 +199,10 @@ public int size() { } @Override - public void save(FileSystem fs, Path path, boolean overwrite) - throws IOException { - serializer().save(fs, path, this, overwrite); + public IOStatistics save(final FileSystem fs, + final Path path, + final JsonSerialization serializer) throws IOException { + return saveFile(fs, path, this, serializer, true); } /** @return the version marker. */ diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PersistentCommitData.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PersistentCommitData.java index dba44b9a01..be2de6cf89 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PersistentCommitData.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PersistentCommitData.java @@ -21,19 +21,33 @@ import java.io.IOException; import java.io.Serializable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FSDataOutputStreamBuilder; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.commit.ValidationFailure; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.util.JsonSerialization; + +import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_CREATE_PERFORMANCE; /** * Class for single/multiple commit data structures. + * The mapreduce hierarchy {@code AbstractManifestData} is a fork + * of this; the Success data JSON format must stay compatible */ -@SuppressWarnings("serial") @InterfaceAudience.Private @InterfaceStability.Unstable -public abstract class PersistentCommitData implements Serializable { +public abstract class PersistentCommitData + implements Serializable, IOStatisticsSource { + private static final Logger LOG = LoggerFactory.getLogger(PersistentCommitData.class); /** * Supported version value: {@value}. @@ -52,18 +66,109 @@ public abstract class PersistentCommitData implements Serializable { * Serialize to JSON and then to a byte array, after performing a * preflight validation of the data to be saved. * @return the data in a persistable form. + * @param serializer serializer to use * @throws IOException serialization problem or validation failure. */ - public abstract byte[] toBytes() throws IOException; + public abstract byte[] toBytes(JsonSerialization serializer) throws IOException; /** * Save to a hadoop filesystem. + * The destination file is overwritten, and on s3a stores the + * performance flag is set to turn off all existence checks and + * parent dir cleanup. + * The assumption here is: the job knows what it is doing. + * * @param fs filesystem * @param path path - * @param overwrite should any existing file be overwritten + * @param serializer serializer to use + * @return IOStats from the output stream. + * * @throws IOException IO exception */ - public abstract void save(FileSystem fs, Path path, boolean overwrite) + public abstract IOStatistics save(FileSystem fs, Path path, JsonSerialization serializer) throws IOException; + /** + * Load an instance from a status, then validate it. + * This uses the openFile() API, which S3A supports for + * faster load and declaring sequential access, always + * @param type of persistent format + * @param fs filesystem + * @param status status of file to load + * @param serializer serializer to use + * @return the loaded instance + * @throws IOException IO failure + * @throws ValidationFailure if the data is invalid + */ + public static T load(FileSystem fs, + FileStatus status, + JsonSerialization serializer) + throws IOException { + Path path = status.getPath(); + LOG.debug("Reading commit data from file {}", path); + T result = serializer.load(fs, path, status); + result.validate(); + return result; + } + + /** + * Save to a file. + * This uses the createFile() API, which S3A supports for + * faster load and declaring sequential access, always + * + * @param type of persistent format + * @param fs filesystem + * @param path path to save to + * @param instance data to save + * @param serializer serializer to use + * @param performance skip all safety check on the write + * + * @return any IOStatistics from the output stream, or null + * + * @throws IOException IO failure + */ + public static IOStatistics saveFile( + final FileSystem fs, + final Path path, + final T instance, + final JsonSerialization serializer, + final boolean performance) + throws IOException { + + FSDataOutputStreamBuilder builder = fs.createFile(path) + .create() + .recursive() + .overwrite(true); + // switch to performance mode + builder.opt(FS_S3A_CREATE_PERFORMANCE, performance); + return saveToStream(path, instance, builder, serializer); + } + + /** + * Save to a file. + * This uses the createFile() API, which S3A supports for + * faster load and declaring sequential access, always + * @param type of persistent format + * @param path path to save to (used for logging) + * @param instance data to save + * @param builder builder already prepared for the write + * @param serializer serializer to use + * @return any IOStatistics from the output stream, or null + * @throws IOException IO failure + */ + public static IOStatistics saveToStream( + final Path path, + final T instance, + final FSDataOutputStreamBuilder builder, + final JsonSerialization serializer) throws IOException { + LOG.debug("saving commit data to file {}", path); + FSDataOutputStream dataOutputStream = builder.build(); + try { + dataOutputStream.write(serializer.toBytes(instance)); + } finally { + dataOutputStream.close(); + } + return dataOutputStream.getIOStatistics(); + } + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java index b53ef75d82..77c3fed11f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java @@ -34,8 +34,6 @@ import com.amazonaws.services.s3.model.PartETag; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.hadoop.util.Preconditions; - import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -43,9 +41,11 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.commit.ValidationFailure; +import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.util.JsonSerialization; +import org.apache.hadoop.util.Preconditions; import static org.apache.hadoop.fs.s3a.commit.CommitUtils.validateCollectionClass; import static org.apache.hadoop.fs.s3a.commit.ValidationFailure.verify; @@ -69,8 +69,8 @@ @SuppressWarnings("unused") @InterfaceAudience.Private @InterfaceStability.Unstable -public class SinglePendingCommit extends PersistentCommitData - implements Iterable, IOStatisticsSource { +public class SinglePendingCommit extends PersistentCommitData + implements Iterable { /** * Serialization ID: {@value}. @@ -141,26 +141,32 @@ public SinglePendingCommit() { * @return a serializer. */ public static JsonSerialization serializer() { - return new JsonSerialization<>(SinglePendingCommit.class, false, true); + return new JsonSerialization<>(SinglePendingCommit.class, false, false); } /** * Load an instance from a file, then validate it. * @param fs filesystem * @param path path + * @param status nullable status of file to load + * @param serDeser serializer; if null use the shared static one. * @return the loaded instance * @throws IOException IO failure * @throws ValidationFailure if the data is invalid */ - public static SinglePendingCommit load(FileSystem fs, Path path) + public static SinglePendingCommit load(FileSystem fs, + Path path, + FileStatus status, + JsonSerialization serDeser) throws IOException { - return load(fs, path, null); + return load(fs, path, serDeser, null); } /** * Load an instance from a file, then validate it. * @param fs filesystem * @param path path + * @param serDeser deserializer * @param status status of file to load or null * @return the loaded instance * @throws IOException IO failure @@ -168,9 +174,12 @@ public static SinglePendingCommit load(FileSystem fs, Path path) */ public static SinglePendingCommit load(FileSystem fs, Path path, + JsonSerialization serDeser, @Nullable FileStatus status) throws IOException { - SinglePendingCommit instance = serializer().load(fs, path, status); + JsonSerialization jsonSerialization = + serDeser != null ? serDeser : serializer(); + SinglePendingCommit instance = jsonSerialization.load(fs, path, status); instance.filename = path.toString(); instance.validate(); return instance; @@ -264,15 +273,16 @@ public String toString() { } @Override - public byte[] toBytes() throws IOException { + public byte[] toBytes(JsonSerialization serializer) throws IOException { validate(); - return serializer().toBytes(this); + return serializer.toBytes(this); } @Override - public void save(FileSystem fs, Path path, boolean overwrite) - throws IOException { - serializer().save(fs, path, this, overwrite); + public IOStatistics save(final FileSystem fs, + final Path path, + final JsonSerialization serializer) throws IOException { + return saveFile(fs, path, this, serializer, true); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SuccessData.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SuccessData.java index 00f196a0b4..0e0a03f587 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SuccessData.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SuccessData.java @@ -30,13 +30,14 @@ import org.slf4j.LoggerFactory; import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.exception.ExceptionUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.commit.ValidationFailure; +import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; -import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.util.JsonSerialization; /** @@ -51,10 +52,24 @@ *
  • Not loadable? Something else.
  • * * - * This is an unstable structure intended for diagnostics and testing. - * Applications reading this data should use/check the {@link #name} field - * to differentiate from any other JSON-based manifest and to identify - * changes in the output format. + * This should be considered public, and MUST stay compatible + * at the JSON format level with that of + * {@code org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData} + *

    + * The JSON format SHOULD be considered public and evolving + * with compatibility across versions. + *

    + * All the Java serialization data is different and may change + * across versions with no stability guarantees other than + * "manifest summaries MAY be serialized between processes with + * the exact same version of this binary on their classpaths." + * That is sufficient for testing in Spark. + *

    + * To aid with Java serialization, the maps and lists are + * exclusively those which serialize well. + * IOStatisticsSnapshot has a lot of complexity in marshalling + * there; this class doesn't worry about concurrent access + * so is simpler. * * Note: to deal with scale issues, the S3A committers do not include any * more than the number of objects listed in @@ -65,8 +80,7 @@ @SuppressWarnings("unused") @InterfaceAudience.Private @InterfaceStability.Unstable -public class SuccessData extends PersistentCommitData - implements IOStatisticsSource { +public class SuccessData extends PersistentCommitData { private static final Logger LOG = LoggerFactory.getLogger(SuccessData.class); @@ -80,7 +94,7 @@ public class SuccessData extends PersistentCommitData /** * Serialization ID: {@value}. */ - private static final long serialVersionUID = 507133045258460083L + VERSION; + private static final long serialVersionUID = 507133045258460084L + VERSION; /** * Name to include in persisted data, so as to differentiate from @@ -92,7 +106,14 @@ public class SuccessData extends PersistentCommitData /** * Name of file; includes version marker. */ - private String name; + private String name = NAME; + + /** + * Did this succeed? + * It is implicitly true in a _SUCCESS file, but if the file + * is also saved to a log dir, then it depends on the outcome + */ + private boolean success = true; /** Timestamp of creation. */ private long timestamp; @@ -142,7 +163,17 @@ public class SuccessData extends PersistentCommitData * IOStatistics. */ @JsonProperty("iostatistics") - private IOStatisticsSnapshot iostats = new IOStatisticsSnapshot(); + private IOStatisticsSnapshot iostatistics = new IOStatisticsSnapshot(); + + /** + * State (committed, aborted). + */ + private String state; + + /** + * Stage: last stage executed. + */ + private String stage; @Override public void validate() throws ValidationFailure { @@ -153,16 +184,17 @@ public void validate() throws ValidationFailure { } @Override - public byte[] toBytes() throws IOException { - return serializer().toBytes(this); + public byte[] toBytes(JsonSerialization serializer) throws IOException { + return serializer.toBytes(this); } @Override - public void save(FileSystem fs, Path path, boolean overwrite) - throws IOException { + public IOStatistics save(final FileSystem fs, + final Path path, + final JsonSerialization serializer) throws IOException { // always set the name field before being saved. name = NAME; - serializer().save(fs, path, this, overwrite); + return saveFile(fs, path, this, serializer, true); } @Override @@ -250,8 +282,8 @@ public static SuccessData load(FileSystem fs, Path path) * Get a JSON serializer for this class. * @return a serializer. */ - private static JsonSerialization serializer() { - return new JsonSerialization<>(SuccessData.class, false, true); + public static JsonSerialization serializer() { + return new JsonSerialization<>(SuccessData.class, false, false); } public String getName() { @@ -371,10 +403,59 @@ public void setJobIdSource(final String jobIdSource) { @Override public IOStatisticsSnapshot getIOStatistics() { - return iostats; + return iostatistics; } public void setIOStatistics(final IOStatisticsSnapshot ioStatistics) { - this.iostats = ioStatistics; + this.iostatistics = ioStatistics; + } + + /** + * Set the success flag. + * @param success did the job succeed? + */ + public void setSuccess(boolean success) { + this.success = success; + } + + /** + * Get the success flag. + * @return did the job succeed? + */ + public boolean getSuccess() { + return success; + } + + public String getState() { + return state; + } + + public void setState(String state) { + this.state = state; + } + + public String getStage() { + return stage; + } + + /** + * Add a diagnostics entry. + * @param key name + * @param value value + */ + public void putDiagnostic(String key, String value) { + diagnostics.put(key, value); + } + + /** + * Note a failure by setting success flag to false, + * then add the exception to the diagnostics. + * @param thrown throwable + */ + public void recordJobFailure(Throwable thrown) { + setSuccess(false); + String stacktrace = ExceptionUtils.getStackTrace(thrown); + diagnostics.put("exception", thrown.toString()); + diagnostics.put("stacktrace", stacktrace); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/package-info.java index 0d574948e9..36eedba83b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/package-info.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/package-info.java @@ -29,12 +29,11 @@ *

  • The summary information saved in the {@code _SUCCESS} file.
  • * * - * There are no guarantees of stability between versions; these are internal - * structures. * * The {@link org.apache.hadoop.fs.s3a.commit.files.SuccessData} file is - * the one visible to callers after a job completes; it is an unstable - * manifest intended for testing only. + * the one visible to callers after a job completes; it is compatible with + * the manifest committer format persisted in + * {@code org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData} * */ @InterfaceAudience.Private diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/AuditContextUpdater.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/AuditContextUpdater.java new file mode 100644 index 0000000000..20024ba601 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/AuditContextUpdater.java @@ -0,0 +1,90 @@ +/* + * 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.s3a.commit.impl; + +import org.apache.hadoop.fs.audit.AuditConstants; +import org.apache.hadoop.fs.audit.CommonAuditContext; +import org.apache.hadoop.fs.s3a.commit.CommitConstants; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.TaskAttemptID; + +import static org.apache.hadoop.fs.audit.CommonAuditContext.currentAuditContext; + +/** + * Class to track/update context information to set + * in threads. + */ +public final class AuditContextUpdater { + + /** + * Job ID. + */ + private final String jobId; + + /** + * Task attempt ID for auditing. + */ + private final String taskAttemptId; + + /** + * Construct. Stores job information + * to attach to thread contexts. + * @param jobContext job/task context. + */ + public AuditContextUpdater(final JobContext jobContext) { + this.jobId = jobContext.getJobID().toString(); + + if (jobContext instanceof TaskAttemptContext) { + // it's a task, extract info for auditing + final TaskAttemptID tid = ((TaskAttemptContext) jobContext).getTaskAttemptID(); + this.taskAttemptId = tid.toString(); + } else { + this.taskAttemptId = null; + } + } + + public AuditContextUpdater(String jobId) { + this.jobId = jobId; + this.taskAttemptId = null; + } + + /** + * Add job/task info to current audit context. + */ + public void updateCurrentAuditContext() { + final CommonAuditContext auditCtx = currentAuditContext(); + auditCtx.put(AuditConstants.PARAM_JOB_ID, jobId); + if (taskAttemptId != null) { + auditCtx.put(AuditConstants.PARAM_TASK_ATTEMPT_ID, taskAttemptId); + } else { + currentAuditContext().remove(CommitConstants.PARAM_TASK_ATTEMPT_ID); + } + + } + + /** + * Remove job/task info from the current audit context. + */ + public void resetCurrentAuditContext() { + currentAuditContext().remove(AuditConstants.PARAM_JOB_ID); + currentAuditContext().remove(CommitConstants.PARAM_TASK_ATTEMPT_ID); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitContext.java new file mode 100644 index 0000000000..8bff165f2e --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitContext.java @@ -0,0 +1,401 @@ +/* + * 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.s3a.commit.impl; + +import java.io.Closeable; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.impl.WeakReferenceThreadMap; +import org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants; +import org.apache.hadoop.fs.s3a.commit.files.PendingSet; +import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.hadoop.util.JsonSerialization; +import org.apache.hadoop.util.Preconditions; +import org.apache.hadoop.util.concurrent.HadoopExecutors; +import org.apache.hadoop.util.concurrent.HadoopThreadPoolExecutor; +import org.apache.hadoop.util.functional.TaskPool; + +import static org.apache.hadoop.fs.s3a.Constants.THREAD_POOL_SHUTDOWN_DELAY_SECONDS; +import static org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter.THREAD_PREFIX; +import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.THREAD_KEEP_ALIVE_TIME; + +/** + * Commit context. + * + * It is used to manage the final commit sequence where files become + * visible. + * + * Once the commit operation has completed, it must be closed. + * It MUST NOT be reused. + * + * Audit integration: job and task attributes are added to the thread local context + * on create, removed on close(). + * + * JSON Serializers are created on demand, on a per thread basis. + * A {@link WeakReferenceThreadMap} is used here; a GC may lose the + * references, but they will recreated as needed. + */ +public final class CommitContext implements Closeable { + + private static final Logger LOG = LoggerFactory.getLogger( + CommitContext.class); + + /** + * The actual commit operations. + */ + private final CommitOperations commitOperations; + + /** + * Job Context. + */ + private final JobContext jobContext; + + /** + * Serializer pool. + */ + + private final WeakReferenceThreadMap> + pendingSetSerializer = + new WeakReferenceThreadMap<>((k) -> PendingSet.serializer(), null); + + private final WeakReferenceThreadMap> + singleCommitSerializer = + new WeakReferenceThreadMap<>((k) -> SinglePendingCommit.serializer(), null); + + /** + * Submitter for per task operations, e.g loading manifests. + */ + private PoolSubmitter outerSubmitter; + + /** + * Submitter for operations within the tasks, + * such as POSTing the final commit operations. + */ + private PoolSubmitter innerSubmitter; + + /** + * Job Configuration. + */ + private final Configuration conf; + + /** + * Job ID. + */ + private final String jobId; + + /** + * Audit context; will be reset when this is closed. + */ + private final AuditContextUpdater auditContextUpdater; + + /** + * Number of committer threads. + */ + private final int committerThreads; + + /** + * Create. + * @param commitOperations commit callbacks + * @param jobContext job context + * @param committerThreads number of commit threads + */ + public CommitContext( + final CommitOperations commitOperations, + final JobContext jobContext, + final int committerThreads) { + this.commitOperations = commitOperations; + this.jobContext = jobContext; + this.conf = jobContext.getConfiguration(); + this.jobId = jobContext.getJobID().toString(); + this.auditContextUpdater = new AuditContextUpdater(jobContext); + this.auditContextUpdater.updateCurrentAuditContext(); + this.committerThreads = committerThreads; + + buildSubmitters(); + } + + /** + * Create for testing. + * This has no job context; instead the values + * are set explicitly. + * @param commitOperations commit callbacks + * @param conf job conf + * @param jobId ID + * @param committerThreads number of commit threads + */ + public CommitContext(final CommitOperations commitOperations, + final Configuration conf, + final String jobId, + final int committerThreads) { + this.commitOperations = commitOperations; + this.jobContext = null; + this.conf = conf; + this.jobId = jobId; + this.auditContextUpdater = new AuditContextUpdater(jobId); + this.auditContextUpdater.updateCurrentAuditContext(); + this.committerThreads = committerThreads; + buildSubmitters(); + } + + /** + * Build the submitters and thread pools if the number of committerThreads + * is greater than zero. + * This should only be called in constructors; it is synchronized to keep + * SpotBugs happy. + */ + private synchronized void buildSubmitters() { + if (committerThreads != 0) { + outerSubmitter = new PoolSubmitter(buildThreadPool(committerThreads)); + } + } + + /** + * Returns an {@link ExecutorService} for parallel tasks. The number of + * threads in the thread-pool is set by fs.s3a.committer.threads. + * If num-threads is 0, this will raise an exception. + * The threads have a lifespan set by + * {@link InternalCommitterConstants#THREAD_KEEP_ALIVE_TIME}. + * When the thread pool is full, the caller runs + * policy takes over. + * @param numThreads thread count, may be negative. + * @return an {@link ExecutorService} for the number of threads + */ + private ExecutorService buildThreadPool( + int numThreads) { + if (numThreads < 0) { + // a negative number means "multiple of available processors" + numThreads = numThreads * -Runtime.getRuntime().availableProcessors(); + } + Preconditions.checkArgument(numThreads > 0, + "Cannot create a thread pool with no threads"); + LOG.debug("creating thread pool of size {}", numThreads); + final ThreadFactory factory = new ThreadFactoryBuilder() + .setDaemon(true) + .setNameFormat(THREAD_PREFIX + jobId + "-%d") + .build(); + return new HadoopThreadPoolExecutor(0, numThreads, + THREAD_KEEP_ALIVE_TIME, + TimeUnit.SECONDS, + new LinkedBlockingQueue<>(), + factory, + new ThreadPoolExecutor.CallerRunsPolicy()); + } + + /** + * Commit the operation, throwing an exception on any failure. + * See {@code CommitOperations#commitOrFail(SinglePendingCommit)}. + * @param commit commit to execute + * @throws IOException on a failure + */ + public void commitOrFail(SinglePendingCommit commit) throws IOException { + commitOperations.commitOrFail(commit); + } + + /** + * Commit a single pending commit; exceptions are caught + * and converted to an outcome. + * See {@link CommitOperations#commit(SinglePendingCommit, String)}. + * @param commit entry to commit + * @param origin origin path/string for outcome text + * @return the outcome + */ + public CommitOperations.MaybeIOE commit(SinglePendingCommit commit, + String origin) { + return commitOperations.commit(commit, origin); + } + + /** + * See {@link CommitOperations#abortSingleCommit(SinglePendingCommit)}. + * @param commit pending commit to abort + * @throws FileNotFoundException if the abort ID is unknown + * @throws IOException on any failure + */ + public void abortSingleCommit(final SinglePendingCommit commit) + throws IOException { + commitOperations.abortSingleCommit(commit); + } + + /** + * See {@link CommitOperations#revertCommit(SinglePendingCommit)}. + * @param commit pending commit + * @throws IOException failure + */ + public void revertCommit(final SinglePendingCommit commit) + throws IOException { + commitOperations.revertCommit(commit); + } + + /** + * See {@link CommitOperations#abortMultipartCommit(String, String)}.. + * @param destKey destination key + * @param uploadId upload to cancel + * @throws FileNotFoundException if the abort ID is unknown + * @throws IOException on any failure + */ + public void abortMultipartCommit( + final String destKey, + final String uploadId) + throws IOException { + commitOperations.abortMultipartCommit(destKey, uploadId); + } + + @Override + public synchronized void close() throws IOException { + + destroyThreadPools(); + auditContextUpdater.resetCurrentAuditContext(); + } + + @Override + public String toString() { + return "CommitContext{}"; + } + + /** + * Job Context. + * @return job context. + */ + public JobContext getJobContext() { + return jobContext; + } + + /** + * Return a submitter. + * If created with 0 threads, this returns null so + * TaskPool knows to run it in the current thread. + * @return a submitter or null + */ + public synchronized TaskPool.Submitter getOuterSubmitter() { + return outerSubmitter; + } + + /** + * Return a submitter. As this pool is used less often, + * create it on demand. + * If created with 0 threads, this returns null so + * TaskPool knows to run it in the current thread. + * @return a submitter or null + */ + public synchronized TaskPool.Submitter getInnerSubmitter() { + if (innerSubmitter == null && committerThreads > 0) { + innerSubmitter = new PoolSubmitter(buildThreadPool(committerThreads)); + } + return innerSubmitter; + } + + /** + * Get a serializer for .pending files. + * @return a serializer. + */ + public JsonSerialization getSinglePendingFileSerializer() { + return singleCommitSerializer.getForCurrentThread(); + } + + /** + * Get a serializer for .pendingset files. + * @return a serializer. + */ + public JsonSerialization getPendingSetSerializer() { + return pendingSetSerializer.getForCurrentThread(); + } + + /** + * Destroy any thread pools; wait for that to finish, + * but don't overreact if it doesn't finish in time. + */ + private synchronized void destroyThreadPools() { + try { + IOUtils.cleanupWithLogger(LOG, outerSubmitter, innerSubmitter); + } finally { + outerSubmitter = null; + innerSubmitter = null; + } + } + + /** + * Job configuration. + * @return configuration (never null) + */ + public Configuration getConf() { + return conf; + } + + /** + * Get the job ID. + * @return job ID. + */ + public String getJobId() { + return jobId; + } + + /** + * Submitter for a given thread pool. + */ + private final class PoolSubmitter implements TaskPool.Submitter, Closeable { + + private ExecutorService executor; + + private PoolSubmitter(ExecutorService executor) { + this.executor = executor; + } + + @Override + public synchronized void close() throws IOException { + if (executor != null) { + HadoopExecutors.shutdown(executor, LOG, + THREAD_POOL_SHUTDOWN_DELAY_SECONDS, TimeUnit.SECONDS); + } + executor = null; + } + + /** + * Forward to the submitter, wrapping in task + * context setting, so as to ensure that all operations + * have job/task attributes. + * @param task task to execute + * @return the future. + */ + @Override + public Future submit(Runnable task) { + return executor.submit(() -> { + auditContextUpdater.updateCurrentAuditContext(); + try { + task.run(); + } finally { + auditContextUpdater.resetCurrentAuditContext(); + } + }); + } + + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java similarity index 82% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java index 840cf8e0f2..0772e143f6 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java @@ -16,24 +16,26 @@ * limitations under the License. */ -package org.apache.hadoop.fs.s3a.commit; +package org.apache.hadoop.fs.s3a.commit.impl; -import java.io.Closeable; import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.UUID; import java.util.stream.Collectors; import java.util.stream.IntStream; +import javax.annotation.Nullable; + import com.amazonaws.services.s3.model.MultipartUpload; import com.amazonaws.services.s3.model.PartETag; import com.amazonaws.services.s3.model.UploadPartRequest; import com.amazonaws.services.s3.model.UploadPartResult; -import org.apache.hadoop.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,25 +49,33 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3AUtils; import org.apache.hadoop.fs.s3a.WriteOperations; +import org.apache.hadoop.fs.s3a.commit.CommitConstants; +import org.apache.hadoop.fs.s3a.commit.PathCommitException; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; import org.apache.hadoop.fs.s3a.commit.files.SuccessData; import org.apache.hadoop.fs.s3a.impl.AbstractStoreOperation; import org.apache.hadoop.fs.s3a.impl.HeaderProcessing; import org.apache.hadoop.fs.s3a.impl.InternalConstants; +import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics; import org.apache.hadoop.fs.statistics.DurationTracker; import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.util.DurationInfo; +import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.util.Progressable; +import org.apache.hadoop.util.functional.TaskPool; import static java.util.Objects.requireNonNull; -import static org.apache.hadoop.fs.s3a.S3AUtils.*; +import static org.apache.hadoop.fs.s3a.S3AUtils.listAndFilter; import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_COMMIT_JOB; import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_MATERIALIZE_FILE; +import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_LOAD_SINGLE_PENDING_FILE; import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_STAGE_FILE_UPLOAD; -import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; +import static org.apache.hadoop.fs.s3a.commit.CommitConstants.XA_MAGIC_MARKER; +import static org.apache.hadoop.fs.s3a.commit.CommitConstants._SUCCESS; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration; import static org.apache.hadoop.util.functional.RemoteIterators.cleanupRemoteIterator; @@ -115,7 +125,7 @@ public class CommitOperations extends AbstractStoreOperation * @throws IOException failure to bind. */ public CommitOperations(S3AFileSystem fs) throws IOException { - this(requireNonNull(fs), fs.newCommitterStatistics()); + this(requireNonNull(fs), fs.newCommitterStatistics(), "/"); } /** @@ -123,10 +133,12 @@ public CommitOperations(S3AFileSystem fs) throws IOException { * the commit operations. * @param fs FS to bind to * @param committerStatistics committer statistics + * @param outputPath destination of work. * @throws IOException failure to bind. */ public CommitOperations(S3AFileSystem fs, - CommitterStatistics committerStatistics) throws IOException { + CommitterStatistics committerStatistics, + String outputPath) throws IOException { super(requireNonNull(fs).createStoreContext()); this.fs = fs; statistics = requireNonNull(committerStatistics); @@ -134,7 +146,7 @@ public CommitOperations(S3AFileSystem fs, writeOperations = fs.createWriteOperationHelper( fs.getAuditSpanSource().createSpan( COMMITTER_COMMIT_JOB.getSymbol(), - "/", null)); + outputPath, null)); } /** @@ -168,7 +180,7 @@ public IOStatistics getIOStatistics() { * @param commit commit to execute * @throws IOException on a failure */ - private void commitOrFail( + public void commitOrFail( final SinglePendingCommit commit) throws IOException { commit(commit, commit.getFilename()).maybeRethrow(); } @@ -180,7 +192,7 @@ private void commitOrFail( * @param origin origin path/string for outcome text * @return the outcome */ - private MaybeIOE commit( + public MaybeIOE commit( final SinglePendingCommit commit, final String origin) { LOG.debug("Committing single commit {}", commit); @@ -225,10 +237,9 @@ private long innerCommit( // finalize the commit writeOperations.commitUpload( commit.getDestinationKey(), - commit.getUploadId(), - toPartEtags(commit.getEtags()), - commit.getLength() - ); + commit.getUploadId(), + toPartEtags(commit.getEtags()), + commit.getLength()); return commit.getLength(); } @@ -236,43 +247,69 @@ private long innerCommit( * Locate all files with the pending suffix under a directory. * @param pendingDir directory * @param recursive recursive listing? - * @return the list of all located entries + * @return iterator of all located entries * @throws IOException if there is a problem listing the path. */ - public List locateAllSinglePendingCommits( + public RemoteIterator locateAllSinglePendingCommits( Path pendingDir, boolean recursive) throws IOException { return listAndFilter(fs, pendingDir, recursive, PENDING_FILTER); } /** - * Load all single pending commits in the directory. + * Load all single pending commits in the directory, using the + * outer submitter. * All load failures are logged and then added to list of files which would * not load. + * * @param pendingDir directory containing commits * @param recursive do a recursive scan? + * @param commitContext commit context + * * @return tuple of loaded entries and those pending files which would * not load/validate. + * * @throws IOException on a failure to list the files. */ public Pair>> - loadSinglePendingCommits(Path pendingDir, boolean recursive) + loadSinglePendingCommits(Path pendingDir, + boolean recursive, + CommitContext commitContext) throws IOException { - List statusList = locateAllSinglePendingCommits( - pendingDir, recursive); - PendingSet commits = new PendingSet( - statusList.size()); - List> failures = new ArrayList<>(1); - for (LocatedFileStatus status : statusList) { - try { - commits.add(SinglePendingCommit.load(fs, status.getPath(), status)); - } catch (IOException e) { - LOG.warn("Failed to load commit file {}", status.getPath(), e); - failures.add(Pair.of(status, e)); - } - } + PendingSet commits = new PendingSet(); + List pendingFiles = Collections.synchronizedList( + new ArrayList<>(1)); + List> failures = Collections.synchronizedList( + new ArrayList<>(1)); + + TaskPool.foreach(locateAllSinglePendingCommits(pendingDir, recursive)) + //. stopOnFailure() + .suppressExceptions(false) + .executeWith(commitContext.getOuterSubmitter()) + .run(status -> { + Path path = status.getPath(); + try { + // load the file + SinglePendingCommit singleCommit = trackDuration(statistics, + COMMITTER_LOAD_SINGLE_PENDING_FILE.getSymbol(), () -> + SinglePendingCommit.load(fs, + path, + status, + commitContext.getSinglePendingFileSerializer())); + // aggregate stats + commits.getIOStatistics() + .aggregate(singleCommit.getIOStatistics()); + // then clear so they aren't marshalled again. + singleCommit.getIOStatistics().clear(); + pendingFiles.add(singleCommit); + } catch (IOException e) { + LOG.warn("Failed to load commit file {}", path, e); + failures.add(Pair.of(status, e)); + } + }); + commits.setCommits(pendingFiles); return Pair.of(commits, failures); } @@ -296,7 +333,7 @@ public IOException makeIOE(String key, Exception ex) { * @throws FileNotFoundException if the abort ID is unknown * @throws IOException on any failure */ - private void abortSingleCommit(SinglePendingCommit commit) + public void abortSingleCommit(SinglePendingCommit commit) throws IOException { String destKey = commit.getDestinationKey(); String origin = commit.getFilename() != null @@ -315,7 +352,7 @@ private void abortSingleCommit(SinglePendingCommit commit) * @throws FileNotFoundException if the abort ID is unknown * @throws IOException on any failure */ - private void abortMultipartCommit(String destKey, String uploadId) + public void abortMultipartCommit(String destKey, String uploadId) throws IOException { try (DurationInfo d = new DurationInfo(LOG, "Aborting commit ID %s to path %s", uploadId, destKey)) { @@ -328,11 +365,13 @@ private void abortMultipartCommit(String destKey, String uploadId) /** * Enumerate all pending files in a dir/tree, abort. * @param pendingDir directory of pending operations + * @param commitContext commit context * @param recursive recurse? * @return the outcome of all the abort operations * @throws IOException if there is a problem listing the path. */ public MaybeIOE abortAllSinglePendingCommits(Path pendingDir, + CommitContext commitContext, boolean recursive) throws IOException { Preconditions.checkArgument(pendingDir != null, "null pendingDir"); @@ -350,12 +389,14 @@ public MaybeIOE abortAllSinglePendingCommits(Path pendingDir, LOG.debug("No files to abort under {}", pendingDir); } while (pendingFiles.hasNext()) { - LocatedFileStatus status = pendingFiles.next(); + final LocatedFileStatus status = pendingFiles.next(); Path pendingFile = status.getPath(); if (pendingFile.getName().endsWith(CommitConstants.PENDING_SUFFIX)) { try { - abortSingleCommit(SinglePendingCommit.load(fs, pendingFile, - status)); + abortSingleCommit(SinglePendingCommit.load(fs, + pendingFile, + status, + commitContext.getSinglePendingFileSerializer())); } catch (FileNotFoundException e) { LOG.debug("listed file already deleted: {}", pendingFile); } catch (IOException | IllegalArgumentException e) { @@ -437,7 +478,7 @@ public void createSuccessMarker(Path outputPath, successData); try (DurationInfo ignored = new DurationInfo(LOG, "Writing success file %s", markerPath)) { - successData.save(fs, markerPath, true); + successData.save(fs, markerPath, SuccessData.serializer()); } } @@ -466,7 +507,7 @@ public void revertCommit(SinglePendingCommit commit) throws IOException { * @return a pending upload entry * @throws IOException failure */ - public SinglePendingCommit uploadFileToPendingCommit(File localFile, + public SinglePendingCommit uploadFileToPendingCommit(File localFile, Path destPath, String partition, long uploadPartSize, @@ -494,7 +535,8 @@ public SinglePendingCommit uploadFileToPendingCommit(File localFile, destPath)) { statistics.commitCreated(); - uploadId = writeOperations.initiateMultiPartUpload(destKey); + uploadId = writeOperations.initiateMultiPartUpload(destKey, + PutObjectOptions.keepingDirs()); long length = localFile.length(); SinglePendingCommit commitData = new SinglePendingCommit(); @@ -592,13 +634,41 @@ public void jobCompleted(boolean success) { } /** - * Begin the final commit. + * Create a commit context for a job or task. + * + * @param context job context * @param path path for all work. + * @param committerThreads thread pool size * @return the commit context to pass in. * @throws IOException failure. */ - public CommitContext initiateCommitOperation(Path path) throws IOException { - return new CommitContext(); + public CommitContext createCommitContext( + JobContext context, + Path path, + int committerThreads) throws IOException { + return new CommitContext(this, context, + committerThreads); + } + + /** + * Create a stub commit context for tests. + * There's no job context. + * @param path path for all work. + * @param jobId job ID; if null a random UUID is generated. + * @param committerThreads number of committer threads. + * @return the commit context to pass in. + * @throws IOException failure. + */ + public CommitContext createCommitContextForTesting( + Path path, @Nullable String jobId, int committerThreads) throws IOException { + final String id = jobId != null + ? jobId + : UUID.randomUUID().toString(); + + return new CommitContext(this, + getStoreContext().getConfiguration(), + id, + committerThreads); } /** @@ -624,98 +694,6 @@ public static Optional extractMagicFileLength(FileSystem fs, Path path) return HeaderProcessing.extractXAttrLongValue(bytes); } - /** - * Commit context. - * - * It is used to manage the final commit sequence where files become - * visible. - * - * This can only be created through {@link #initiateCommitOperation(Path)}. - * - * Once the commit operation has completed, it must be closed. - * It must not be reused. - */ - public final class CommitContext implements Closeable { - - - /** - * Create. - */ - private CommitContext() { - } - - /** - * Commit the operation, throwing an exception on any failure. - * See {@link CommitOperations#commitOrFail(SinglePendingCommit)}. - * @param commit commit to execute - * @throws IOException on a failure - */ - public void commitOrFail(SinglePendingCommit commit) throws IOException { - CommitOperations.this.commitOrFail(commit); - } - - /** - * Commit a single pending commit; exceptions are caught - * and converted to an outcome. - * See {@link CommitOperations#commit(SinglePendingCommit, String)}. - * @param commit entry to commit - * @param origin origin path/string for outcome text - * @return the outcome - */ - public MaybeIOE commit(SinglePendingCommit commit, - String origin) { - return CommitOperations.this.commit(commit, origin); - } - - /** - * See {@link CommitOperations#abortSingleCommit(SinglePendingCommit)}. - * @param commit pending commit to abort - * @throws FileNotFoundException if the abort ID is unknown - * @throws IOException on any failure - */ - public void abortSingleCommit(final SinglePendingCommit commit) - throws IOException { - CommitOperations.this.abortSingleCommit(commit); - } - - /** - * See {@link CommitOperations#revertCommit(SinglePendingCommit)}. - * @param commit pending commit - * @throws IOException failure - */ - public void revertCommit(final SinglePendingCommit commit) - throws IOException { - CommitOperations.this.revertCommit(commit); - } - - /** - * See {@link CommitOperations#abortMultipartCommit(String, String)}.. - * @param destKey destination key - * @param uploadId upload to cancel - * @throws FileNotFoundException if the abort ID is unknown - * @throws IOException on any failure - */ - public void abortMultipartCommit( - final String destKey, - final String uploadId) - throws IOException { - CommitOperations.this.abortMultipartCommit(destKey, uploadId); - } - - @Override - public void close() throws IOException { - } - - @Override - public String toString() { - final StringBuilder sb = new StringBuilder( - "CommitContext{"); - sb.append('}'); - return sb.toString(); - } - - } - /** * A holder for a possible IOException; the call {@link #maybeRethrow()} * will throw any exception passed into the constructor, and be a no-op @@ -788,5 +766,4 @@ public static MaybeIOE of(IOException ex) { } } - } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitUtilsWithMR.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitUtilsWithMR.java similarity index 74% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitUtilsWithMR.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitUtilsWithMR.java index 9e5ee860e8..c38ab2e9ba 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitUtilsWithMR.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitUtilsWithMR.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.fs.s3a.commit; +package org.apache.hadoop.fs.s3a.commit.impl; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -68,31 +68,67 @@ public static int getAppAttemptId(JobContext context) { /** * Compute the "magic" path for a job attempt. * @param jobUUID unique Job ID. + * @param appAttemptId the ID of the application attempt for this job. * @param dest the final output directory * @return the path to store job attempt data. */ - public static Path getMagicJobAttemptPath(String jobUUID, Path dest) { - return new Path(getMagicJobAttemptsPath(dest), - formatAppAttemptDir(jobUUID)); + public static Path getMagicJobAttemptPath(String jobUUID, + int appAttemptId, + Path dest) { + return new Path( + getMagicJobAttemptsPath(dest), + formatAppAttemptDir(jobUUID, appAttemptId)); } /** - * Format the application attempt directory. + * Compute the "magic" path for a job. * @param jobUUID unique Job ID. - * @return the directory name for the application attempt + * @param dest the final output directory + * @return the path to store job attempt data. */ - public static String formatAppAttemptDir(String jobUUID) { + public static Path getMagicJobPath(String jobUUID, + Path dest) { + return new Path( + getMagicJobAttemptsPath(dest), + formatJobDir(jobUUID)); + } + + /** + * Build the name of the job directory, without + * app attempt. + * This is the path to use for cleanup. + * @param jobUUID unique Job ID. + * @return the directory name for the job + */ + public static String formatJobDir( + String jobUUID) { return String.format("job-%s", jobUUID); } + /** + * Build the name of the job attempt directory. + * @param jobUUID unique Job ID. + * @param appAttemptId the ID of the application attempt for this job. + * @return the directory tree for the application attempt + */ + public static String formatAppAttemptDir( + String jobUUID, + int appAttemptId) { + return formatJobDir(jobUUID) + String.format("/%02d", appAttemptId); + } + /** * Compute the path where the output of magic task attempts are stored. * @param jobUUID unique Job ID. * @param dest destination of work + * @param appAttemptId the ID of the application attempt for this job. * @return the path where the output of magic task attempts are stored. */ - public static Path getMagicTaskAttemptsPath(String jobUUID, Path dest) { - return new Path(getMagicJobAttemptPath(jobUUID, dest), "tasks"); + public static Path getMagicTaskAttemptsPath( + String jobUUID, + Path dest, + int appAttemptId) { + return new Path(getMagicJobAttemptPath(jobUUID, appAttemptId, dest), "tasks"); } /** @@ -115,6 +151,8 @@ public static Path getMagicTaskAttemptPath(TaskAttemptContext context, /** * Get the base Magic attempt path, without any annotations to mark relative * references. + * If there is an app attempt property in the context configuration, that + * is included. * @param context task context. * @param jobUUID unique Job ID. * @param dest The output path to commit work into @@ -123,8 +161,9 @@ public static Path getMagicTaskAttemptPath(TaskAttemptContext context, public static Path getBaseMagicTaskAttemptPath(TaskAttemptContext context, String jobUUID, Path dest) { - return new Path(getMagicTaskAttemptsPath(jobUUID, dest), - String.valueOf(context.getTaskAttemptID())); + return new Path( + getMagicTaskAttemptsPath(jobUUID, dest, getAppAttemptId(context)), + String.valueOf(context.getTaskAttemptID())); } /** @@ -132,12 +171,13 @@ public static Path getBaseMagicTaskAttemptPath(TaskAttemptContext context, * This data is not magic * @param jobUUID unique Job ID. * @param out output directory of job + * @param appAttemptId the ID of the application attempt for this job. * @return the path to store temporary job attempt data. */ public static Path getTempJobAttemptPath(String jobUUID, - Path out) { + Path out, final int appAttemptId) { return new Path(new Path(out, TEMP_DATA), - formatAppAttemptDir(jobUUID)); + formatAppAttemptDir(jobUUID, appAttemptId)); } /** @@ -150,7 +190,7 @@ public static Path getTempJobAttemptPath(String jobUUID, public static Path getTempTaskAttemptPath(TaskAttemptContext context, final String jobUUID, Path out) { return new Path( - getTempJobAttemptPath(jobUUID, out), + getTempJobAttemptPath(jobUUID, out, getAppAttemptId(context)), String.valueOf(context.getTaskAttemptID())); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/package-info.java new file mode 100644 index 0000000000..b4977f34d5 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/package-info.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Internal classes which make use of mapreduce code. + * These MUST NOT be referred to in production code except + * in org.apache.hadoop.fs.s3a.commit classes which are only + * used within job/task committers. + */ + +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.fs.s3a.commit.impl; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java index 7ea4e88308..c85571a194 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java @@ -26,20 +26,26 @@ import com.amazonaws.services.s3.model.PartETag; import com.amazonaws.services.s3.model.PutObjectRequest; -import org.apache.hadoop.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.Retries; import org.apache.hadoop.fs.s3a.WriteOperationHelper; import org.apache.hadoop.fs.s3a.commit.PutTracker; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; +import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; +import org.apache.hadoop.fs.s3a.statistics.PutTrackerStatistics; import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.util.Preconditions; +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_MAGIC_MARKER_PUT; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.X_HEADER_MAGIC_MARKER; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation; /** * Put tracker for Magic commits. @@ -57,6 +63,7 @@ public class MagicCommitTracker extends PutTracker { private final WriteOperationHelper writer; private final String bucket; private static final byte[] EMPTY = new byte[0]; + private final PutTrackerStatistics trackerStatistics; /** * Magic commit tracker. @@ -66,19 +73,22 @@ public class MagicCommitTracker extends PutTracker { * @param destKey key for the destination * @param pendingsetKey key of the pendingset file * @param writer writer instance to use for operations; includes audit span + * @param trackerStatistics tracker statistics */ public MagicCommitTracker(Path path, String bucket, String originalDestKey, String destKey, String pendingsetKey, - WriteOperationHelper writer) { + WriteOperationHelper writer, + PutTrackerStatistics trackerStatistics) { super(destKey); this.bucket = bucket; this.path = path; this.originalDestKey = originalDestKey; this.pendingPartKey = pendingsetKey; this.writer = writer; + this.trackerStatistics = requireNonNull(trackerStatistics); LOG.info("File {} is written as magic file to path {}", path, destKey); } @@ -126,6 +136,19 @@ public boolean aboutToComplete(String uploadId, Preconditions.checkArgument(!parts.isEmpty(), "No uploaded parts to save"); + // put a 0-byte file with the name of the original under-magic path + // Add the final file length as a header + // this is done before the task commit, so its duration can be + // included in the statistics + Map headers = new HashMap<>(); + headers.put(X_HEADER_MAGIC_MARKER, Long.toString(bytesWritten)); + PutObjectRequest originalDestPut = writer.createPutObjectRequest( + originalDestKey, + new ByteArrayInputStream(EMPTY), + 0, + new PutObjectOptions(true, null, headers)); + upload(originalDestPut); + // build the commit summary SinglePendingCommit commitData = new SinglePendingCommit(); commitData.touch(System.currentTimeMillis()); @@ -138,7 +161,8 @@ public boolean aboutToComplete(String uploadId, commitData.bindCommitData(parts); commitData.setIOStatistics( new IOStatisticsSnapshot(iostatistics)); - byte[] bytes = commitData.toBytes(); + + byte[] bytes = commitData.toBytes(SinglePendingCommit.serializer()); LOG.info("Uncommitted data pending to file {};" + " commit metadata for {} parts in {}. size: {} byte(s)", path.toUri(), parts.size(), pendingPartKey, bytesWritten); @@ -148,19 +172,20 @@ public boolean aboutToComplete(String uploadId, pendingPartKey, new ByteArrayInputStream(bytes), bytes.length, null); - writer.uploadObject(put); - - // Add the final file length as a header - Map headers = new HashMap<>(); - headers.put(X_HEADER_MAGIC_MARKER, Long.toString(bytesWritten)); - // now put a 0-byte file with the name of the original under-magic path - PutObjectRequest originalDestPut = writer.createPutObjectRequest( - originalDestKey, - new ByteArrayInputStream(EMPTY), - 0, - headers); - writer.uploadObject(originalDestPut); + upload(put); return false; + + } + /** + * PUT an object. + * @param request the request + * @throws IOException on problems + */ + @Retries.RetryTranslated + private void upload(PutObjectRequest request) throws IOException { + trackDurationOfInvocation(trackerStatistics, + COMMITTER_MAGIC_MARKER_PUT.getSymbol(), () -> + writer.uploadObject(request, PutObjectOptions.keepingDirs())); } @Override diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java index c1ecd7d6b9..007e9b3709 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java @@ -32,11 +32,12 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.Invoker; import org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter; -import org.apache.hadoop.fs.s3a.commit.CommitOperations; +import org.apache.hadoop.fs.s3a.commit.impl.CommitContext; +import org.apache.hadoop.fs.s3a.commit.impl.CommitOperations; import org.apache.hadoop.fs.s3a.commit.CommitConstants; -import org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; +import org.apache.hadoop.fs.s3a.commit.impl.CommitUtilsWithMR; import org.apache.hadoop.fs.statistics.IOStatisticsLogging; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.TaskAttemptContext; @@ -45,9 +46,10 @@ import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.TASK_ATTEMPT_ID; +import static org.apache.hadoop.fs.s3a.commit.CommitConstants.TEMP_DATA; import static org.apache.hadoop.fs.s3a.commit.CommitUtils.*; import static org.apache.hadoop.fs.s3a.commit.MagicCommitPaths.*; -import static org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR.*; +import static org.apache.hadoop.fs.s3a.commit.impl.CommitUtilsWithMR.*; import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatistics; /** @@ -100,25 +102,28 @@ public void setupJob(JobContext context) throws IOException { try (DurationInfo d = new DurationInfo(LOG, "Setup Job %s", jobIdString(context))) { super.setupJob(context); - Path jobAttemptPath = getJobAttemptPath(context); - getDestinationFS(jobAttemptPath, - context.getConfiguration()).mkdirs(jobAttemptPath); + Path jobPath = getJobPath(); + final FileSystem destFS = getDestinationFS(jobPath, + context.getConfiguration()); + destFS.delete(jobPath, true); + destFS.mkdirs(jobPath); } } /** * Get the list of pending uploads for this job attempt, by listing * all .pendingset files in the job attempt directory. - * @param context job context + * @param commitContext job context * @return a list of pending commits. * @throws IOException Any IO failure */ protected ActiveCommit listPendingUploadsToCommit( - JobContext context) + CommitContext commitContext) throws IOException { FileSystem fs = getDestFS(); - return ActiveCommit.fromStatusList(fs, - listAndFilter(fs, getJobAttemptPath(context), false, + return ActiveCommit.fromStatusIterator(fs, + listAndFilter(fs, getJobAttemptPath(commitContext.getJobContext()), + false, CommitOperations.PENDINGSET_FILTER)); } @@ -126,11 +131,16 @@ protected ActiveCommit listPendingUploadsToCommit( * Delete the magic directory. */ public void cleanupStagingDirs() { - Path path = magicSubdir(getOutputPath()); + final Path out = getOutputPath(); + Path path = magicSubdir(out); try(DurationInfo ignored = new DurationInfo(LOG, true, "Deleting magic directory %s", path)) { Invoker.ignoreIOExceptions(LOG, "cleanup magic directory", path.toString(), () -> deleteWithWarning(getDestFS(), path, true)); + // and the job temp directory with manifests + Invoker.ignoreIOExceptions(LOG, "cleanup job directory", path.toString(), + () -> deleteWithWarning(getDestFS(), + new Path(out, TEMP_DATA), true)); } } @@ -146,13 +156,8 @@ public void cleanupStagingDirs() { @Override public boolean needsTaskCommit(TaskAttemptContext context) throws IOException { - Path taskAttemptPath = getTaskAttemptPath(context); - try (DurationInfo d = new DurationInfo(LOG, - "needsTaskCommit task %s", context.getTaskAttemptID())) { - return taskAttemptPath.getFileSystem( - context.getConfiguration()) - .exists(taskAttemptPath); - } + // return true as a dir was created here in setup; + return true; } @Override @@ -167,9 +172,9 @@ public void commitTask(TaskAttemptContext context) throws IOException { throw e; } finally { // delete the task attempt so there's no possibility of a second attempt + // incurs a LIST, a bulk DELETE and maybe a parent dir creation, however + // as it happens during task commit, it should be off the critical path. deleteTaskAttemptPathQuietly(context); - destroyThreadPool(); - resetCommonContext(); } getCommitOperations().taskCompleted(true); LOG.debug("aggregate statistics\n{}", @@ -191,43 +196,48 @@ private PendingSet innerCommitTask( Path taskAttemptPath = getTaskAttemptPath(context); // load in all pending commits. CommitOperations actions = getCommitOperations(); - Pair>> - loaded = actions.loadSinglePendingCommits( - taskAttemptPath, true); - PendingSet pendingSet = loaded.getKey(); - List> failures = loaded.getValue(); - if (!failures.isEmpty()) { - // At least one file failed to load - // revert all which did; report failure with first exception - LOG.error("At least one commit file could not be read: failing"); - abortPendingUploads(context, pendingSet.getCommits(), true); - throw failures.get(0).getValue(); - } - // patch in IDs - String jobId = getUUID(); - String taskId = String.valueOf(context.getTaskAttemptID()); - for (SinglePendingCommit commit : pendingSet.getCommits()) { - commit.setJobId(jobId); - commit.setTaskId(taskId); - } - pendingSet.putExtraData(TASK_ATTEMPT_ID, taskId); - pendingSet.setJobId(jobId); - Path jobAttemptPath = getJobAttemptPath(context); - TaskAttemptID taskAttemptID = context.getTaskAttemptID(); - Path taskOutcomePath = new Path(jobAttemptPath, - taskAttemptID.getTaskID().toString() + - CommitConstants.PENDINGSET_SUFFIX); - LOG.info("Saving work of {} to {}", taskAttemptID, taskOutcomePath); - LOG.debug("task statistics\n{}", - IOStatisticsLogging.demandStringifyIOStatisticsSource(pendingSet)); - try { - // We will overwrite if there exists a pendingSet file already - pendingSet.save(getDestFS(), taskOutcomePath, true); - } catch (IOException e) { - LOG.warn("Failed to save task commit data to {} ", - taskOutcomePath, e); - abortPendingUploads(context, pendingSet.getCommits(), true); - throw e; + PendingSet pendingSet; + try (CommitContext commitContext = initiateTaskOperation(context)) { + Pair>> + loaded = actions.loadSinglePendingCommits( + taskAttemptPath, true, commitContext); + pendingSet = loaded.getKey(); + List> failures = loaded.getValue(); + if (!failures.isEmpty()) { + // At least one file failed to load + // revert all which did; report failure with first exception + LOG.error("At least one commit file could not be read: failing"); + abortPendingUploads(commitContext, pendingSet.getCommits(), true); + throw failures.get(0).getValue(); + } + // patch in IDs + String jobId = getUUID(); + String taskId = String.valueOf(context.getTaskAttemptID()); + for (SinglePendingCommit commit : pendingSet.getCommits()) { + commit.setJobId(jobId); + commit.setTaskId(taskId); + } + pendingSet.putExtraData(TASK_ATTEMPT_ID, taskId); + pendingSet.setJobId(jobId); + Path jobAttemptPath = getJobAttemptPath(context); + TaskAttemptID taskAttemptID = context.getTaskAttemptID(); + Path taskOutcomePath = new Path(jobAttemptPath, + taskAttemptID.getTaskID().toString() + + CommitConstants.PENDINGSET_SUFFIX); + LOG.info("Saving work of {} to {}", taskAttemptID, taskOutcomePath); + LOG.debug("task statistics\n{}", + IOStatisticsLogging.demandStringifyIOStatisticsSource(pendingSet)); + try { + // We will overwrite if there exists a pendingSet file already + pendingSet.save(getDestFS(), + taskOutcomePath, + commitContext.getPendingSetSerializer()); + } catch (IOException e) { + LOG.warn("Failed to save task commit data to {} ", + taskOutcomePath, e); + abortPendingUploads(commitContext, pendingSet.getCommits(), true); + throw e; + } } return pendingSet; } @@ -246,25 +256,35 @@ private PendingSet innerCommitTask( public void abortTask(TaskAttemptContext context) throws IOException { Path attemptPath = getTaskAttemptPath(context); try (DurationInfo d = new DurationInfo(LOG, - "Abort task %s", context.getTaskAttemptID())) { - getCommitOperations().abortAllSinglePendingCommits(attemptPath, true); + "Abort task %s", context.getTaskAttemptID()); + CommitContext commitContext = initiateTaskOperation(context)) { + getCommitOperations().abortAllSinglePendingCommits(attemptPath, + commitContext, + true); } finally { deleteQuietly( attemptPath.getFileSystem(context.getConfiguration()), attemptPath, true); - destroyThreadPool(); - resetCommonContext(); } } + /** + * Compute the path under which all job attempts will be placed. + * @return the path to store job attempt data. + */ + @Override + protected Path getJobPath() { + return getMagicJobPath(getUUID(), getOutputPath()); + } + /** * Compute the path where the output of a given job attempt will be placed. * For the magic committer, the path includes the job UUID. * @param appAttemptId the ID of the application attempt for this job. * @return the path to store job attempt data. */ - protected Path getJobAttemptPath(int appAttemptId) { - return getMagicJobAttemptPath(getUUID(), getOutputPath()); + protected final Path getJobAttemptPath(int appAttemptId) { + return getMagicJobAttemptPath(getUUID(), appAttemptId, getOutputPath()); } /** @@ -274,12 +294,12 @@ protected Path getJobAttemptPath(int appAttemptId) { * @param context the context of the task attempt. * @return the path where a task attempt should be stored. */ - public Path getTaskAttemptPath(TaskAttemptContext context) { + public final Path getTaskAttemptPath(TaskAttemptContext context) { return getMagicTaskAttemptPath(context, getUUID(), getOutputPath()); } @Override - protected Path getBaseTaskAttemptPath(TaskAttemptContext context) { + protected final Path getBaseTaskAttemptPath(TaskAttemptContext context) { return getBaseMagicTaskAttemptPath(context, getUUID(), getOutputPath()); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/DirectoryStagingCommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/DirectoryStagingCommitter.java index 1a5a63c940..99683db984 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/DirectoryStagingCommitter.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/DirectoryStagingCommitter.java @@ -30,6 +30,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathExistsException; import org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants; +import org.apache.hadoop.fs.s3a.commit.impl.CommitContext; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.TaskAttemptContext; @@ -98,17 +99,18 @@ public void setupJob(JobContext context) throws IOException { * Pre-commit actions for a job. * Here: look at the conflict resolution mode and choose * an action based on the current policy. - * @param context job context + * @param commitContext commit context * @param pending pending commits * @throws IOException any failure */ @Override public void preCommitJob( - final JobContext context, + final CommitContext commitContext, final ActiveCommit pending) throws IOException { + final JobContext context = commitContext.getJobContext(); // see if the files can be loaded. - super.preCommitJob(context, pending); + super.preCommitJob(commitContext, pending); Path outputPath = getOutputPath(); FileSystem fs = getDestFS(); Configuration fsConf = fs.getConf(); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/PartitionedStagingCommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/PartitionedStagingCommitter.java index 214c7abdc7..5d1a20e424 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/PartitionedStagingCommitter.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/PartitionedStagingCommitter.java @@ -32,12 +32,13 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.commit.PathCommitException; -import org.apache.hadoop.fs.s3a.commit.Tasks; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; +import org.apache.hadoop.fs.s3a.commit.files.PersistentCommitData; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; -import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.fs.s3a.commit.impl.CommitContext; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.util.DurationInfo; +import org.apache.hadoop.util.functional.TaskPool; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.COMMITTER_NAME_PARTITIONED; @@ -89,7 +90,8 @@ public String toString() { @Override protected int commitTaskInternal(TaskAttemptContext context, - List taskOutput) throws IOException { + List taskOutput, + CommitContext commitContext) throws IOException { Path attemptPath = getTaskAttemptPath(context); Set partitions = Paths.getPartitions(attemptPath, taskOutput); @@ -109,7 +111,7 @@ protected int commitTaskInternal(TaskAttemptContext context, } } } - return super.commitTaskInternal(context, taskOutput); + return super.commitTaskInternal(context, taskOutput, commitContext); } /** @@ -121,13 +123,13 @@ protected int commitTaskInternal(TaskAttemptContext context, *
  • APPEND: allowed.; no need to check.
  • *
  • REPLACE deletes all existing partitions.
  • * - * @param context job context + * @param commitContext commit context * @param pending the pending operations * @throws IOException any failure */ @Override public void preCommitJob( - final JobContext context, + final CommitContext commitContext, final ActiveCommit pending) throws IOException { FileSystem fs = getDestFS(); @@ -135,7 +137,7 @@ public void preCommitJob( // enforce conflict resolution Configuration fsConf = fs.getConf(); boolean shouldPrecheckPendingFiles = true; - switch (getConflictResolutionMode(context, fsConf)) { + switch (getConflictResolutionMode(commitContext.getJobContext(), fsConf)) { case FAIL: // FAIL checking is done on the task side, so this does nothing break; @@ -144,17 +146,17 @@ public void preCommitJob( break; case REPLACE: // identify and replace the destination partitions - replacePartitions(context, pending); + replacePartitions(commitContext, pending); // and so there is no need to do another check. shouldPrecheckPendingFiles = false; break; default: throw new PathCommitException("", getRole() + ": unknown conflict resolution mode: " - + getConflictResolutionMode(context, fsConf)); + + getConflictResolutionMode(commitContext.getJobContext(), fsConf)); } if (shouldPrecheckPendingFiles) { - precommitCheckPendingFiles(context, pending); + precommitCheckPendingFiles(commitContext, pending); } } @@ -176,17 +178,16 @@ public void preCommitJob( * } * * - * @param context job context + * @param commitContext commit context * @param pending the pending operations * @throws IOException any failure */ private void replacePartitions( - final JobContext context, + final CommitContext commitContext, final ActiveCommit pending) throws IOException { Map partitions = new ConcurrentHashMap<>(); FileSystem sourceFS = pending.getSourceFS(); - Tasks.Submitter submitter = buildSubmitter(context); try (DurationInfo ignored = new DurationInfo(LOG, "Replacing partitions")) { @@ -194,13 +195,15 @@ private void replacePartitions( // for a marginal optimisation, the previous parent is tracked, so // if a task writes many files to the same dir, the synchronized map // is updated only once. - Tasks.foreach(pending.getSourceFiles()) + TaskPool.foreach(pending.getSourceFiles()) .stopOnFailure() .suppressExceptions(false) - .executeWith(submitter) + .executeWith(commitContext.getOuterSubmitter()) .run(status -> { - PendingSet pendingSet = PendingSet.load(sourceFS, - status); + PendingSet pendingSet = PersistentCommitData.load( + sourceFS, + status, + commitContext.getPendingSetSerializer()); Path lastParent = null; for (SinglePendingCommit commit : pendingSet.getCommits()) { Path parent = commit.destinationPath().getParent(); @@ -213,10 +216,10 @@ private void replacePartitions( } // now do the deletes FileSystem fs = getDestFS(); - Tasks.foreach(partitions.keySet()) + TaskPool.foreach(partitions.keySet()) .stopOnFailure() .suppressExceptions(false) - .executeWith(submitter) + .executeWith(commitContext.getOuterSubmitter()) .run(partitionPath -> { LOG.debug("{}: removing partition path to be replaced: " + getRole(), partitionPath); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java index 121ea7f851..36eae012de 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java @@ -39,24 +39,25 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter; import org.apache.hadoop.fs.s3a.commit.CommitConstants; -import org.apache.hadoop.fs.s3a.commit.CommitOperations; import org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants; -import org.apache.hadoop.fs.s3a.commit.Tasks; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; +import org.apache.hadoop.fs.s3a.commit.impl.CommitContext; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter; import org.apache.hadoop.util.DurationInfo; +import org.apache.hadoop.util.functional.TaskPool; -import static org.apache.hadoop.util.Preconditions.*; +import static java.util.Objects.requireNonNull; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.Invoker.*; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; import static org.apache.hadoop.fs.s3a.commit.CommitUtils.*; -import static org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR.*; +import static org.apache.hadoop.fs.s3a.commit.impl.CommitUtilsWithMR.*; import static org.apache.hadoop.util.functional.RemoteIterators.cleanupRemoteIterator; +import static org.apache.hadoop.util.functional.RemoteIterators.toList; /** * Committer based on the contributed work of the @@ -111,7 +112,7 @@ public class StagingCommitter extends AbstractS3ACommitter { public StagingCommitter(Path outputPath, TaskAttemptContext context) throws IOException { super(outputPath, context); - this.constructorOutputPath = checkNotNull(getOutputPath(), "output path"); + this.constructorOutputPath = requireNonNull(getOutputPath(), "output path"); Configuration conf = getConf(); this.uploadPartSize = conf.getLongBytes( MULTIPART_SIZE, DEFAULT_MULTIPART_SIZE); @@ -121,8 +122,8 @@ public StagingCommitter(Path outputPath, setWorkPath(buildWorkPath(context, getUUID())); this.wrappedCommitter = createWrappedCommitter(context, conf); setOutputPath(constructorOutputPath); - Path finalOutputPath = getOutputPath(); - checkNotNull(finalOutputPath, "Output path cannot be null"); + Path finalOutputPath = requireNonNull(getOutputPath(), + "Output path cannot be null"); S3AFileSystem fs = getS3AFileSystem(finalOutputPath, context.getConfiguration(), false); s3KeyPrefix = fs.pathToKey(finalOutputPath); @@ -243,10 +244,18 @@ private static Path getJobAttemptPath(int appAttemptId, Path out) { @Override protected Path getJobAttemptPath(int appAttemptId) { - return new Path(getPendingJobAttemptsPath(commitsDirectory), + return new Path(getJobPath(), String.valueOf(appAttemptId)); } + /** + * Compute the path under which all job attempts will be placed. + * @return the path to store job attempt data. + */ + protected Path getJobPath() { + return getPendingJobAttemptsPath(commitsDirectory); + } + /** * Compute the path where the output of pending task attempts are stored. * @param context the context of the job with pending tasks. @@ -275,7 +284,7 @@ public static Path getTaskAttemptPath(TaskAttemptContext context, Path out) { * @return the location of pending job attempts. */ private static Path getPendingJobAttemptsPath(Path out) { - checkNotNull(out, "Null 'out' path"); + requireNonNull(out, "Null 'out' path"); return new Path(out, TEMPORARY); } @@ -296,12 +305,12 @@ public Path getCommittedTaskPath(TaskAttemptContext context) { * @param context task context */ private static void validateContext(TaskAttemptContext context) { - checkNotNull(context, "null context"); - checkNotNull(context.getTaskAttemptID(), + requireNonNull(context, "null context"); + requireNonNull(context.getTaskAttemptID(), "null task attempt ID"); - checkNotNull(context.getTaskAttemptID().getTaskID(), + requireNonNull(context.getTaskAttemptID().getTaskID(), "null task ID"); - checkNotNull(context.getTaskAttemptID().getJobID(), + requireNonNull(context.getTaskAttemptID().getJobID(), "null job ID"); } @@ -342,14 +351,13 @@ protected List getTaskOutput(TaskAttemptContext context) throws IOException { // get files on the local FS in the attempt path - Path attemptPath = getTaskAttemptPath(context); - checkNotNull(attemptPath, - "No attemptPath path in {}", this); + Path attemptPath = requireNonNull(getTaskAttemptPath(context), + "No attemptPath path"); LOG.debug("Scanning {} for files to commit", attemptPath); - return listAndFilter(getTaskAttemptFilesystem(context), - attemptPath, true, HIDDEN_FILE_FILTER); + return toList(listAndFilter(getTaskAttemptFilesystem(context), + attemptPath, true, HIDDEN_FILE_FILTER)); } /** @@ -425,46 +433,46 @@ public void setupJob(JobContext context) throws IOException { /** * Get the list of pending uploads for this job attempt. - * @param context job context + * @param commitContext job context * @return a list of pending uploads. * @throws IOException Any IO failure */ @Override protected ActiveCommit listPendingUploadsToCommit( - JobContext context) + CommitContext commitContext) throws IOException { - return listPendingUploads(context, false); + return listPendingUploads(commitContext, false); } /** * Get the list of pending uploads for this job attempt, swallowing * exceptions. - * @param context job context + * @param commitContext commit context * @return a list of pending uploads. If an exception was swallowed, * then this may not match the actual set of pending operations * @throws IOException shouldn't be raised, but retained for the compiler */ protected ActiveCommit listPendingUploadsToAbort( - JobContext context) throws IOException { - return listPendingUploads(context, true); + CommitContext commitContext) throws IOException { + return listPendingUploads(commitContext, true); } /** * Get the list of pending uploads for this job attempt. - * @param context job context + * @param commitContext commit context * @param suppressExceptions should exceptions be swallowed? * @return a list of pending uploads. If exceptions are being swallowed, * then this may not match the actual set of pending operations * @throws IOException Any IO failure which wasn't swallowed. */ protected ActiveCommit listPendingUploads( - JobContext context, boolean suppressExceptions) throws IOException { + CommitContext commitContext, boolean suppressExceptions) throws IOException { try (DurationInfo ignored = new DurationInfo(LOG, "Listing pending uploads")) { - Path wrappedJobAttemptPath = getJobAttemptPath(context); + Path wrappedJobAttemptPath = getJobAttemptPath(commitContext.getJobContext()); final FileSystem attemptFS = wrappedJobAttemptPath.getFileSystem( - context.getConfiguration()); - return ActiveCommit.fromStatusList(attemptFS, + commitContext.getConf()); + return ActiveCommit.fromStatusIterator(attemptFS, listAndFilter(attemptFS, wrappedJobAttemptPath, false, HIDDEN_FILE_FILTER)); @@ -491,27 +499,39 @@ public void cleanupStagingDirs() { } } + /** + * Staging committer cleanup includes calling wrapped committer's + * cleanup method, and removing all destination paths in the final + * filesystem. + * @param commitContext commit context + * @param suppressExceptions should exceptions be suppressed? + * @throws IOException IO failures if exceptions are not suppressed. + */ @Override @SuppressWarnings("deprecation") - protected void cleanup(JobContext context, + protected void cleanup(CommitContext commitContext, boolean suppressExceptions) throws IOException { maybeIgnore(suppressExceptions, "Cleanup wrapped committer", - () -> wrappedCommitter.cleanupJob(context)); + () -> wrappedCommitter.cleanupJob( + commitContext.getJobContext())); maybeIgnore(suppressExceptions, "Delete destination paths", - () -> deleteDestinationPaths(context)); - super.cleanup(context, suppressExceptions); + () -> deleteDestinationPaths( + commitContext.getJobContext())); + super.cleanup(commitContext, suppressExceptions); } @Override - protected void abortJobInternal(JobContext context, + protected void abortJobInternal(CommitContext commitContext, boolean suppressExceptions) throws IOException { String r = getRole(); + JobContext context = commitContext.getJobContext(); boolean failed = false; try (DurationInfo d = new DurationInfo(LOG, "%s: aborting job in state %s ", r, jobIdString(context))) { - ActiveCommit pending = listPendingUploadsToAbort(context); - abortPendingUploads(context, pending, suppressExceptions, true); + ActiveCommit pending = listPendingUploadsToAbort(commitContext); + abortPendingUploads(commitContext, + pending, suppressExceptions, true); } catch (FileNotFoundException e) { // nothing to list LOG.debug("No job directory to read uploads from"); @@ -519,7 +539,7 @@ protected void abortJobInternal(JobContext context, failed = true; maybeIgnore(suppressExceptions, "aborting job", e); } finally { - super.abortJobInternal(context, failed || suppressExceptions); + super.abortJobInternal(commitContext, failed || suppressExceptions); } } @@ -590,17 +610,16 @@ public boolean needsTaskCommit(TaskAttemptContext context) @Override public void commitTask(TaskAttemptContext context) throws IOException { try (DurationInfo d = new DurationInfo(LOG, - "%s: commit task %s", getRole(), context.getTaskAttemptID())) { - int count = commitTaskInternal(context, getTaskOutput(context)); + "%s: commit task %s", getRole(), context.getTaskAttemptID()); + CommitContext commitContext + = initiateTaskOperation(context)) { + int count = commitTaskInternal(context, getTaskOutput(context), commitContext); LOG.info("{}: upload file count: {}", getRole(), count); } catch (IOException e) { LOG.error("{}: commit of task {} failed", getRole(), context.getTaskAttemptID(), e); getCommitOperations().taskCompleted(false); throw e; - } finally { - destroyThreadPool(); - resetCommonContext(); } getCommitOperations().taskCompleted(true); } @@ -610,11 +629,13 @@ public void commitTask(TaskAttemptContext context) throws IOException { * writing a pending entry for them. * @param context task context * @param taskOutput list of files from the output + * @param commitContext commit context * @return number of uploads committed. * @throws IOException IO Failures. */ protected int commitTaskInternal(final TaskAttemptContext context, - List taskOutput) + List taskOutput, + CommitContext commitContext) throws IOException { LOG.debug("{}: commitTaskInternal", getRole()); Configuration conf = context.getConfiguration(); @@ -649,10 +670,10 @@ protected int commitTaskInternal(final TaskAttemptContext context, pendingCommits.putExtraData(TASK_ATTEMPT_ID, context.getTaskAttemptID().toString()); try { - Tasks.foreach(taskOutput) + TaskPool.foreach(taskOutput) .stopOnFailure() .suppressExceptions(false) - .executeWith(buildSubmitter(context)) + .executeWith(commitContext.getOuterSubmitter()) .run(stat -> { Path path = stat.getPath(); File localFile = new File(path.toUri().getPath()); @@ -676,13 +697,14 @@ protected int commitTaskInternal(final TaskAttemptContext context, } // save the data - // although overwrite=false, there's still a risk of > 1 entry being - // committed if the FS doesn't have create-no-overwrite consistency. + // overwrite any existing file, so whichever task attempt + // committed last wins. LOG.debug("Saving {} pending commit(s)) to file {}", pendingCommits.size(), commitsAttemptPath); - pendingCommits.save(commitsFS, commitsAttemptPath, false); + pendingCommits.save(commitsFS, commitsAttemptPath, + commitContext.getPendingSetSerializer()); threw = false; } finally { @@ -690,12 +712,11 @@ protected int commitTaskInternal(final TaskAttemptContext context, LOG.error( "{}: Exception during commit process, aborting {} commit(s)", getRole(), commits.size()); - try(CommitOperations.CommitContext commitContext - = initiateCommitOperation(); - DurationInfo ignored = new DurationInfo(LOG, + try(DurationInfo ignored = new DurationInfo(LOG, "Aborting %s uploads", commits.size())) { - Tasks.foreach(commits) + TaskPool.foreach(commits) .suppressExceptions() + .executeWith(commitContext.getOuterSubmitter()) .run(commitContext::abortSingleCommit); } deleteTaskAttemptPathQuietly(context); @@ -738,9 +759,6 @@ public void abortTask(TaskAttemptContext context) throws IOException { LOG.error("{}: exception when aborting task {}", getRole(), context.getTaskAttemptID(), e); throw e; - } finally { - destroyThreadPool(); - resetCommonContext(); } } @@ -859,16 +877,16 @@ public static String getConfictModeOption(JobContext context, * Pre-commit actions for a job. * Loads all the pending files to verify they can be loaded * and parsed. - * @param context job context + * @param commitContext commit context * @param pending pending commits * @throws IOException any failure */ @Override public void preCommitJob( - final JobContext context, + CommitContext commitContext, final ActiveCommit pending) throws IOException { // see if the files can be loaded. - precommitCheckPendingFiles(context, pending); + precommitCheckPendingFiles(commitContext, pending); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CreateFileBuilder.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CreateFileBuilder.java new file mode 100644 index 0000000000..0392afac59 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CreateFileBuilder.java @@ -0,0 +1,263 @@ +/* + * 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.s3a.impl; + +import java.io.IOException; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import javax.annotation.Nonnull; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CreateFlag; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FSDataOutputStreamBuilder; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.s3a.Constants; +import org.apache.hadoop.util.Progressable; + +import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_CREATE_HEADER; +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.CREATE_FILE_KEYS; + +/** + * Builder used in create file; takes a callback to the operation + * to create the file. + * Is non-recursive unless explicitly changed. + */ +public class CreateFileBuilder extends + FSDataOutputStreamBuilder { + + /** + * Flag set to create with overwrite. + */ + public static final EnumSet CREATE_OVERWRITE_FLAGS = + EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE); + + /** + * Flag set to create without overwrite. + */ + public static final EnumSet CREATE_NO_OVERWRITE_FLAGS = + EnumSet.of(CreateFlag.CREATE); + + /** + * Classic create file option set: overwriting. + */ + public static final CreateFileOptions OPTIONS_CREATE_FILE_OVERWRITE = + new CreateFileOptions(CREATE_OVERWRITE_FLAGS, true, false, null); + + /** + * Classic create file option set: no overwrite. + */ + public static final CreateFileOptions OPTIONS_CREATE_FILE_NO_OVERWRITE = + new CreateFileOptions(CREATE_NO_OVERWRITE_FLAGS, true, false, null); + + /** + * Callback interface. + */ + private final CreateFileBuilderCallbacks callbacks; + + /** + * Constructor. + * @param fileSystem fs; used by superclass. + * @param path qualified path to create + * @param callbacks callbacks. + */ + public CreateFileBuilder( + @Nonnull final FileSystem fileSystem, + @Nonnull final Path path, + @Nonnull final CreateFileBuilderCallbacks callbacks) { + + super(fileSystem, path); + this.callbacks = callbacks; + } + + @Override + public CreateFileBuilder getThisBuilder() { + return this; + } + + @Override + public FSDataOutputStream build() throws IOException { + Path path = getPath(); + + final Configuration options = getOptions(); + final Map headers = new HashMap<>(); + final Set mandatoryKeys = getMandatoryKeys(); + final Set keysToValidate = new HashSet<>(); + + // pick up all headers from the mandatory list and strip them before + // validating the keys + String headerPrefix = FS_S3A_CREATE_HEADER + "."; + final int prefixLen = headerPrefix.length(); + mandatoryKeys.stream().forEach(key -> { + if (key.startsWith(headerPrefix) && key.length() > prefixLen) { + headers.put(key.substring(prefixLen), options.get(key)); + } else { + keysToValidate.add(key); + } + }); + + rejectUnknownMandatoryKeys(keysToValidate, CREATE_FILE_KEYS, "for " + path); + + // and add any optional headers + getOptionalKeys().stream() + .filter(key -> key.startsWith(headerPrefix) && key.length() > prefixLen) + .forEach(key -> headers.put(key.substring(prefixLen), options.get(key))); + + + EnumSet flags = getFlags(); + if (flags.contains(CreateFlag.APPEND)) { + throw new UnsupportedOperationException("Append is not supported"); + } + if (!flags.contains(CreateFlag.CREATE) && + !flags.contains(CreateFlag.OVERWRITE)) { + throw new PathIOException(path.toString(), + "Must specify either create or overwrite"); + } + + final boolean performance = + options.getBoolean(Constants.FS_S3A_CREATE_PERFORMANCE, false); + return callbacks.createFileFromBuilder( + path, + getProgress(), + new CreateFileOptions(flags, isRecursive(), performance, headers)); + + } + + /** + * Pass flags down. + * @param flags input flags. + * @return this builder. + */ + public CreateFileBuilder withFlags(EnumSet flags) { + if (flags.contains(CreateFlag.CREATE)) { + create(); + } + if (flags.contains(CreateFlag.APPEND)) { + append(); + } + overwrite(flags.contains(CreateFlag.OVERWRITE)); + return this; + } + + /** + * make the flag getter public. + * @return creation flags. + */ + public EnumSet getFlags() { + return super.getFlags(); + } + + /** + * Callbacks for creating the file. + */ + public interface CreateFileBuilderCallbacks { + + /** + * Create a file from the builder. + * @param path path to file + * @param progress progress callback + * @param options options for the file + * @return the stream + * @throws IOException any IO problem + */ + FSDataOutputStream createFileFromBuilder( + Path path, + Progressable progress, + CreateFileOptions options) throws IOException; + } + + /** + * Create file options as built from the builder set or the classic + * entry point. + */ + public static final class CreateFileOptions { + + /** + * creation flags. + * create parent dirs? + * progress callback. + * performance flag. + */ + private final EnumSet flags; + + /** + * create parent dirs? + */ + private final boolean recursive; + + /** + * performance flag. + */ + private final boolean performance; + + /** + * Headers; may be null. + */ + private final Map headers; + + /** + * @param flags creation flags + * @param recursive create parent dirs? + * @param performance performance flag + * @param headers nullable header map. + */ + public CreateFileOptions( + final EnumSet flags, + final boolean recursive, + final boolean performance, + final Map headers) { + this.flags = flags; + this.recursive = recursive; + this.performance = performance; + this.headers = headers; + } + + @Override + public String toString() { + return "CreateFileOptions{" + + "flags=" + flags + + ", recursive=" + recursive + + ", performance=" + performance + + ", headers=" + headers + + '}'; + } + + public EnumSet getFlags() { + return flags; + } + + public boolean isRecursive() { + return recursive; + } + + public boolean isPerformance() { + return performance; + } + + public Map getHeaders() { + return headers; + } + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java index 49b9feeb6f..6e4946dfb5 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java @@ -18,7 +18,9 @@ package org.apache.hadoop.fs.s3a.impl; +import java.util.Arrays; import java.util.Collections; +import java.util.HashSet; import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -160,4 +162,12 @@ private InternalConstants() { * will go through the AccessPoint. */ public static final String ARN_BUCKET_OPTION = "fs.s3a.bucket.%s.accesspoint.arn"; + + /** + * The known keys used in a createFile call. + */ + public static final Set CREATE_FILE_KEYS = + Collections.unmodifiableSet( + new HashSet<>(Arrays.asList(Constants.FS_S3A_CREATE_PERFORMANCE))); + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MkdirOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MkdirOperation.java index 2b9a0e89b1..98a91b1881 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MkdirOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MkdirOperation.java @@ -41,6 +41,18 @@ * It performs the directory listing probe ahead of the simple object HEAD * call for this reason -the object is the failure mode which SHOULD NOT * be encountered on normal execution. + * + * Magic paths are handled specially + *
      + *
    • The only path check is for a directory already existing there.
    • + *
    • No ancestors are checked
    • + *
    • Parent markers are never deleted, irrespective of FS settings
    • + *
    + * As a result, irrespective of depth, the operations performed are only + *
      + *
    1. One LIST
    2. + *
    3. If needed, one PUT
    4. + *
    */ public class MkdirOperation extends ExecutingStoreOperation { @@ -51,13 +63,21 @@ public class MkdirOperation extends ExecutingStoreOperation { private final MkdirCallbacks callbacks; + /** + * Should checks for ancestors existing be skipped? + * This flag is set when working with magic directories. + */ + private final boolean isMagicPath; + public MkdirOperation( final StoreContext storeContext, final Path dir, - final MkdirCallbacks callbacks) { + final MkdirCallbacks callbacks, + final boolean isMagicPath) { super(storeContext); this.dir = dir; this.callbacks = callbacks; + this.isMagicPath = isMagicPath; } /** @@ -77,6 +97,14 @@ public Boolean execute() throws IOException { return true; } + // get the file status of the path. + // this is done even for a magic path, to avoid always issuing PUT + // requests. Doing that without a check wouild seem to be an + // optimization, but it is not because + // 1. PUT is slower than HEAD + // 2. Write capacity is less than read capacity on a shard + // 3. It adds needless entries in versioned buckets, slowing + // down subsequent operations. FileStatus fileStatus = getPathStatusExpectingDir(dir); if (fileStatus != null) { if (fileStatus.isDirectory()) { @@ -85,7 +113,17 @@ public Boolean execute() throws IOException { throw new FileAlreadyExistsException("Path is a file: " + dir); } } - // dir, walk up tree + // file status was null + + // is the path magic? + // If so, we declare success without looking any further + if (isMagicPath) { + // Create the marker file immediately, + // and don't delete markers + callbacks.createFakeDirectory(dir, true); + return true; + } + // Walk path to root, ensuring closest ancestor is a directory, not file Path fPart = dir.getParent(); try { @@ -110,14 +148,15 @@ public Boolean execute() throws IOException { LOG.info("mkdirs({}}: Access denied when looking" + " for parent directory {}; skipping checks", dir, fPart); - LOG.debug("{}", e.toString(), e); + LOG.debug("{}", e, e); } // if we get here there is no directory at the destination. // so create one. - String key = getStoreContext().pathToKey(dir); - // Create the marker file, maybe delete the parent entries - callbacks.createFakeDirectory(key); + + // Create the marker file, delete the parent entries + // if the filesystem isn't configured to retain them + callbacks.createFakeDirectory(dir, false); return true; } @@ -140,15 +179,21 @@ private S3AFileStatus probePathStatusOrNull(final Path path, /** * Get the status of a path -optimized for paths * where there is a directory marker or child entries. + * + * Under a magic path, there's no check for a file, + * just the listing. + * * @param path path to probe. + * * @return the status + * * @throws IOException failure */ private S3AFileStatus getPathStatusExpectingDir(final Path path) throws IOException { S3AFileStatus status = probePathStatusOrNull(path, StatusProbeEnum.DIRECTORIES); - if (status == null) { + if (status == null && !isMagicPath) { status = probePathStatusOrNull(path, StatusProbeEnum.FILE); } @@ -174,10 +219,15 @@ S3AFileStatus probePathStatus(Path path, /** * Create a fake directory, always ending in "/". * Retry policy: retrying; translated. - * @param key name of directory object. + * the keepMarkers flag controls whether or not markers + * are automatically kept (this is set when creating + * directories under a magic path, always) + * @param dir dir to create + * @param keepMarkers always keep markers + * * @throws IOException IO failure */ @Retries.RetryTranslated - void createFakeDirectory(String key) throws IOException; + void createFakeDirectory(Path dir, boolean keepMarkers) throws IOException; } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/PutObjectOptions.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/PutObjectOptions.java new file mode 100644 index 0000000000..e14285a1ca --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/PutObjectOptions.java @@ -0,0 +1,104 @@ +/* + * 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.s3a.impl; + +import java.util.Map; +import javax.annotation.Nullable; + +/** + * Extensible structure for options when putting/writing objects. + */ +public final class PutObjectOptions { + + /** + * Can the PUT operation skip marker deletion? + */ + private final boolean keepMarkers; + + /** + * Storage class, if not null. + */ + private final String storageClass; + + /** + * Headers; may be null. + */ + private final Map headers; + + /** + * Constructor. + * @param keepMarkers Can the PUT operation skip marker deletion? + * @param storageClass Storage class, if not null. + * @param headers Headers; may be null. + */ + public PutObjectOptions( + final boolean keepMarkers, + @Nullable final String storageClass, + @Nullable final Map headers) { + this.keepMarkers = keepMarkers; + this.storageClass = storageClass; + this.headers = headers; + } + + /** + * Get the marker retention flag. + * @return true if markers are to be retained. + */ + public boolean isKeepMarkers() { + return keepMarkers; + } + + /** + * Headers for the put/post request. + * @return headers or null. + */ + public Map getHeaders() { + return headers; + } + + @Override + public String toString() { + return "PutObjectOptions{" + + "keepMarkers=" + keepMarkers + + ", storageClass='" + storageClass + '\'' + + '}'; + } + + private static final PutObjectOptions KEEP_DIRS = new PutObjectOptions(true, + null, null); + private static final PutObjectOptions DELETE_DIRS = new PutObjectOptions(false, + null, null); + + /** + * Get the options to keep directories. + * @return an instance which keeps dirs + */ + public static PutObjectOptions keepingDirs() { + return KEEP_DIRS; + } + + /** + * Get the options to delete directory markers. + * @return an instance which deletes dirs + */ + public static PutObjectOptions deletingDirs() { + return DELETE_DIRS; + } + +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java index 5a693c6c3d..1e6629f9c7 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java @@ -23,7 +23,9 @@ import java.io.InputStream; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.Optional; +import javax.annotation.Nullable; import com.amazonaws.AmazonWebServiceRequest; import com.amazonaws.services.s3.model.AbortMultipartUploadRequest; @@ -362,15 +364,19 @@ protected void copyEncryptionParameters( * Adds the ACL, storage class and metadata * @param key key of object * @param metadata metadata header + * @param options options for the request, including headers * @param srcfile source file * @return the request */ @Override public PutObjectRequest newPutObjectRequest(String key, - ObjectMetadata metadata, File srcfile) { + ObjectMetadata metadata, + final PutObjectOptions options, + File srcfile) { Preconditions.checkNotNull(srcfile); PutObjectRequest putObjectRequest = new PutObjectRequest(getBucket(), key, srcfile); + maybeSetMetadata(options, metadata); setOptionalPutRequestParameters(putObjectRequest); putObjectRequest.setCannedAcl(cannedACL); if (storageClass != null) { @@ -386,15 +392,18 @@ public PutObjectRequest newPutObjectRequest(String key, * operation. * @param key key of object * @param metadata metadata header + * @param options options for the request * @param inputStream source data. * @return the request */ @Override public PutObjectRequest newPutObjectRequest(String key, ObjectMetadata metadata, + @Nullable final PutObjectOptions options, InputStream inputStream) { Preconditions.checkNotNull(inputStream); Preconditions.checkArgument(isNotEmpty(key), "Null/empty key"); + maybeSetMetadata(options, metadata); PutObjectRequest putObjectRequest = new PutObjectRequest(getBucket(), key, inputStream, metadata); setOptionalPutRequestParameters(putObjectRequest); @@ -418,7 +427,7 @@ public int read() throws IOException { final ObjectMetadata md = createObjectMetadata(0L, true); md.setContentType(HeaderProcessing.CONTENT_TYPE_X_DIRECTORY); PutObjectRequest putObjectRequest = - newPutObjectRequest(key, md, im); + newPutObjectRequest(key, md, null, im); return putObjectRequest; } @@ -444,11 +453,14 @@ public AbortMultipartUploadRequest newAbortMultipartUploadRequest( @Override public InitiateMultipartUploadRequest newMultipartUploadRequest( - String destKey) { + final String destKey, + @Nullable final PutObjectOptions options) { + final ObjectMetadata objectMetadata = newObjectMetadata(-1); + maybeSetMetadata(options, objectMetadata); final InitiateMultipartUploadRequest initiateMPURequest = new InitiateMultipartUploadRequest(getBucket(), destKey, - newObjectMetadata(-1)); + objectMetadata); initiateMPURequest.setCannedACL(getCannedACL()); if (getStorageClass() != null) { initiateMPURequest.withStorageClass(getStorageClass()); @@ -601,6 +613,23 @@ public void setEncryptionSecrets(final EncryptionSecrets secrets) { encryptionSecrets = secrets; } + /** + * Set the metadata from the options if the options are not + * null and the metadata contains headers. + * @param options options for the request + * @param objectMetadata metadata to patch + */ + private void maybeSetMetadata( + @Nullable PutObjectOptions options, + final ObjectMetadata objectMetadata) { + if (options != null) { + Map headers = options.getHeaders(); + if (headers != null) { + objectMetadata.setUserMetadata(headers); + } + } + } + /** * Create a builder. * @return new builder. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java index db6beaff5b..3a6a04e5e7 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java @@ -123,7 +123,8 @@ public CompletableFuture startUpload( String key = context.pathToKey(dest); return context.submit(new CompletableFuture<>(), () -> { - String uploadId = writeOperations.initiateMultiPartUpload(key); + String uploadId = writeOperations.initiateMultiPartUpload(key, + PutObjectOptions.keepingDirs()); statistics.uploadStarted(); return BBUploadHandle.from(ByteBuffer.wrap( uploadId.getBytes(Charsets.UTF_8))); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/BlockOutputStreamStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/BlockOutputStreamStatistics.java index 772b965d4f..bd1466b2a4 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/BlockOutputStreamStatistics.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/BlockOutputStreamStatistics.java @@ -25,7 +25,8 @@ * Block output stream statistics. */ public interface BlockOutputStreamStatistics extends Closeable, - S3AStatisticInterface { + S3AStatisticInterface, + PutTrackerStatistics { /** * Block is queued for upload. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/CommitterStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/CommitterStatistics.java index fd232a058d..53c25b0c65 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/CommitterStatistics.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/CommitterStatistics.java @@ -18,6 +18,8 @@ package org.apache.hadoop.fs.s3a.statistics; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; + /** * Statistics for S3A committers. */ @@ -63,4 +65,10 @@ public interface CommitterStatistics * @param success success flag */ void jobCompleted(boolean success); + + /** + * Return the writeable IOStatisticsStore. + * @return the statistics + */ + IOStatisticsStore getIOStatistics(); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/PutTrackerStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/PutTrackerStatistics.java new file mode 100644 index 0000000000..b422e0c9d5 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/PutTrackerStatistics.java @@ -0,0 +1,29 @@ +/* + * 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.s3a.statistics; + +/** + * Interface for PUT tracking. + * It is subclassed by {@link BlockOutputStreamStatistics}, + * so that operations performed by the PutTracker update + * the stream statistics. + * Having a separate interface helps isolate operations. + */ +public interface PutTrackerStatistics extends S3AStatisticInterface { +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/EmptyS3AStatisticsContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/EmptyS3AStatisticsContext.java index f618270798..5c0995e41b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/EmptyS3AStatisticsContext.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/EmptyS3AStatisticsContext.java @@ -33,6 +33,7 @@ import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk; import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.DurationTracker; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.emptyStatistics; import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.stubDurationTracker; @@ -137,6 +138,7 @@ private static class EmptyS3AStatisticImpl implements public DurationTracker trackDuration(String key, long count) { return stubDurationTracker(); } + } /** @@ -381,6 +383,11 @@ public void taskCompleted(final boolean success) { @Override public void jobCompleted(final boolean success) { } + + @Override + public IOStatisticsStore getIOStatistics() { + return null; + } } private static final class EmptyBlockOutputStreamStatistics diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing.md index 7c00462735..8ccc36cf83 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing.md @@ -226,18 +226,25 @@ If any of the field values were `null`, the field is omitted. | `cm` | Command | `S3GuardTool$BucketInfo` | | `fs` | FileSystem ID | `af5943a9-b6f6-4eec-9c58-008982fc492a` | | `id` | Span ID | `3c0d9b7e-2a63-43d9-a220-3c574d768ef3-3` | -| `ji` | Job ID | `(Generated by query engine)` | +| `ji` | Job ID (S3A committer)| `(Generated by query engine)` | | `op` | Filesystem API call | `op_rename` | | `p1` | Path 1 of operation | `s3a://alice-london/path1` | | `p2` | Path 2 of operation | `s3a://alice-london/path2` | | `pr` | Principal | `alice` | | `ps` | Unique process UUID | `235865a0-d399-4696-9978-64568db1b51c` | +| `ta` | Task Attempt ID (S3A committer) | | | `t0` | Thread 0: thread span was created in | `100` | | `t1` | Thread 1: thread this operation was executed in | `200` | | `ts` | Timestamp (UTC epoch millis) | `1617116985923` | +_Notes_ -Thread IDs are from the current thread in the JVM. +* Thread IDs are from the current thread in the JVM, so can be compared to those in````````` + Log4J logs. They are never unique. +* Task Attempt/Job IDs are only ever set during operations involving the S3A committers, specifically + all operations excecuted by the committer. + Operations executed in the same thread as the committer's instantiation _may_ also report the + IDs, even if they are unrelated to the actual task. Consider them "best effort". ```java Long.toString(Thread.currentThread().getId()) diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/committers.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/committers.md index b19f30f1a3..cfeff28d54 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/committers.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/committers.md @@ -549,7 +549,7 @@ Conflict management is left to the execution engine itself. | `fs.s3a.buffer.dir` | Local filesystem directory for data being written and/or staged. | `${env.LOCAL_DIRS:-${hadoop.tmp.dir}}/s3a` | | `fs.s3a.committer.magic.enabled` | Enable "magic committer" support in the filesystem. | `true` | | `fs.s3a.committer.abort.pending.uploads` | list and abort all pending uploads under the destination path when the job is committed or aborted. | `true` | -| `fs.s3a.committer.threads` | Number of threads in committers for parallel operations on files. | 8 | +| `fs.s3a.committer.threads` | Number of threads in committers for parallel operations on files.| -4 | | `fs.s3a.committer.generate.uuid` | Generate a Job UUID if none is passed down from Spark | `false` | | `fs.s3a.committer.require.uuid` |Require the Job UUID to be passed down from Spark | `false` | @@ -587,10 +587,15 @@ Conflict management is left to the execution engine itself. fs.s3a.committer.threads - 8 + -4 Number of threads in committers for parallel operations on files - (upload, commit, abort, delete...) + (upload, commit, abort, delete...). + Two thread pools this size are created, one for the outer + task-level parallelism, and one for parallel execution + within tasks (POSTs to commit individual uploads) + If the value is negative, it is inverted and then multiplied + by the number of cores in the CPU. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java index 27c70b2b21..dae6312d48 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java @@ -18,8 +18,6 @@ package org.apache.hadoop.fs.s3a; - -import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; @@ -369,67 +367,6 @@ public void testNeedEmptyDirectoryProbeRequiresList() throws Throwable { fs.s3GetFileStatus(new Path("/something"), "/something", StatusProbeEnum.HEAD_ONLY, true)); } - @Test - public void testCreateCost() throws Throwable { - describe("Test file creation cost"); - Path testFile = methodPath(); - // when overwrite is false, the path is checked for existence. - create(testFile, false, - CREATE_FILE_NO_OVERWRITE); - // but when true: only the directory checks take place. - create(testFile, true, CREATE_FILE_OVERWRITE); - } - - @Test - public void testCreateCostFileExists() throws Throwable { - describe("Test cost of create file failing with existing file"); - Path testFile = file(methodPath()); - - // now there is a file there, an attempt with overwrite == false will - // fail on the first HEAD. - interceptOperation(FileAlreadyExistsException.class, "", - FILE_STATUS_FILE_PROBE, - () -> file(testFile, false)); - } - - @Test - public void testCreateCostDirExists() throws Throwable { - describe("Test cost of create file failing with existing dir"); - Path testFile = dir(methodPath()); - - // now there is a file there, an attempt with overwrite == false will - // fail on the first HEAD. - interceptOperation(FileAlreadyExistsException.class, "", - GET_FILE_STATUS_ON_DIR_MARKER, - () -> file(testFile, false)); - } - - /** - * Use the builder API. - * This always looks for a parent unless the caller says otherwise. - */ - @Test - public void testCreateBuilder() throws Throwable { - describe("Test builder file creation cost"); - Path testFile = methodPath(); - dir(testFile.getParent()); - - // builder defaults to looking for parent existence (non-recursive) - buildFile(testFile, false, false, - GET_FILE_STATUS_FNFE // destination file - .plus(FILE_STATUS_DIR_PROBE)); // parent dir - // recursive = false and overwrite=true: - // only make sure the dest path isn't a directory. - buildFile(testFile, true, true, - FILE_STATUS_DIR_PROBE); - - // now there is a file there, an attempt with overwrite == false will - // fail on the first HEAD. - interceptOperation(FileAlreadyExistsException.class, "", - GET_FILE_STATUS_ON_FILE, - () -> buildFile(testFile, false, true, - GET_FILE_STATUS_ON_FILE)); - } @Test public void testCostOfGlobStatus() throws Throwable { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java index b4d2527a46..2d29282ad0 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java @@ -36,8 +36,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonPathCapabilities; import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; import org.apache.hadoop.fs.store.audit.AuditSpan; import org.apache.hadoop.fs.store.EtagChecksum; import org.apache.hadoop.test.LambdaTestUtils; @@ -102,18 +102,6 @@ public void testCreateNonRecursiveSuccess() throws IOException { assertIsFile(shouldWork); } - @Test(expected = FileNotFoundException.class) - public void testCreateNonRecursiveNoParent() throws IOException { - createNonRecursive(path("/recursive/node")); - } - - @Test(expected = FileAlreadyExistsException.class) - public void testCreateNonRecursiveParentIsFile() throws IOException { - Path parent = path("/file.txt"); - touch(getFileSystem(), parent); - createNonRecursive(new Path(parent, "fail")); - } - @Test public void testPutObjectDirect() throws Throwable { final S3AFileSystem fs = getFileSystem(); @@ -126,7 +114,7 @@ public void testPutObjectDirect() throws Throwable { new ByteArrayInputStream("PUT".getBytes()), metadata); LambdaTestUtils.intercept(IllegalStateException.class, - () -> fs.putObjectDirect(put)); + () -> fs.putObjectDirect(put, PutObjectOptions.keepingDirs())); assertPathDoesNotExist("put object was created", path); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java index b597460bfd..2eb35daab4 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java @@ -40,7 +40,11 @@ import org.apache.hadoop.fs.s3a.audit.AuditTestSupport; import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets; import org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase; +import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; import org.apache.hadoop.fs.s3a.impl.RequestFactoryImpl; +import org.apache.hadoop.fs.s3a.impl.StoreContext; +import org.apache.hadoop.fs.s3a.impl.StoreContextBuilder; +import org.apache.hadoop.fs.s3a.impl.StubContextAccessor; import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics; import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext; import org.apache.hadoop.fs.statistics.DurationTrackerFactory; @@ -211,7 +215,11 @@ public boolean exists(Path f) throws IOException { } @Override - void finishedWrite(String key, long length, String eTag, String versionId) { + void finishedWrite(String key, + long length, + String eTag, + String versionId, + final PutObjectOptions putOptions) { } @@ -377,11 +385,29 @@ public CommitterStatistics newCommitterStatistics() { @Override public void operationRetried(Exception ex) { - /** no-op */ + /* no-op */ } @Override protected DurationTrackerFactory getDurationTrackerFactory() { return stubDurationTrackerFactory(); } + + /** + * Build an immutable store context. + * If called while the FS is being initialized, + * some of the context will be incomplete. + * new store context instances should be created as appropriate. + * @return the store context of this FS. + */ + public StoreContext createStoreContext() { + return new StoreContextBuilder().setFsURI(getUri()) + .setBucket(getBucket()) + .setConfiguration(getConf()) + .setUsername(getUsername()) + .setAuditor(getAuditor()) + .setContextAccessors(new StubContextAccessor(getBucket())) + .build(); + } + } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MultipartTestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MultipartTestUtils.java index 04c2b2a09b..9ea33cf69c 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MultipartTestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MultipartTestUtils.java @@ -22,6 +22,7 @@ import com.amazonaws.services.s3.model.PartETag; import com.amazonaws.services.s3.model.UploadPartRequest; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; import org.apache.hadoop.fs.store.audit.AuditSpan; import org.apache.hadoop.io.IOUtils; @@ -78,7 +79,7 @@ public static IdKey createPartUpload(S3AFileSystem fs, String key, int len, WriteOperationHelper writeHelper = fs.getWriteOperationHelper(); byte[] data = dataset(len, 'a', 'z'); InputStream in = new ByteArrayInputStream(data); - String uploadId = writeHelper.initiateMultiPartUpload(key); + String uploadId = writeHelper.initiateMultiPartUpload(key, PutObjectOptions.keepingDirs()); UploadPartRequest req = writeHelper.newUploadPartRequest(key, uploadId, partNo, len, in, null, 0L); PartETag partEtag = writeHelper.uploadPart(req).getPartETag(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java index 47ff2f326e..a6269c4376 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java @@ -245,4 +245,10 @@ public interface S3ATestConstants { * used. */ int KMS_KEY_GENERATION_REQUEST_PARAMS_BYTES_WRITTEN = 94; + + /** + * Build directory property. + * Value: {@value}. + */ + String PROJECT_BUILD_DIRECTORY_PROPERTY = "project.build.directory"; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java index 21ad7f87d6..48cb52c5ac 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java @@ -35,8 +35,6 @@ import org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding; import org.apache.hadoop.fs.s3a.auth.MarshalledCredentials; import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets; -import org.apache.hadoop.fs.s3a.commit.CommitConstants; - import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; import org.apache.hadoop.fs.s3a.impl.ContextAccessors; import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; @@ -577,6 +575,19 @@ public static Configuration prepareTestConfiguration(final Configuration conf) { return conf; } + /** + * build dir. + * @return the directory for the project's build, as set by maven, + * falling back to pwd + "target" if running from an IDE; + */ + public static File getProjectBuildDir() { + String propval = System.getProperty(PROJECT_BUILD_DIRECTORY_PROPERTY); + if (StringUtils.isEmpty(propval)) { + propval = "target"; + } + return new File(propval).getAbsoluteFile(); + } + /** * Clear any Hadoop credential provider path. * This is needed if people's test setups switch to credential providers, @@ -1301,18 +1312,6 @@ public static long lsR(FileSystem fileSystem, Path path, boolean recursive) public static final DateFormat LISTING_FORMAT = new SimpleDateFormat( "yyyy-MM-dd HH:mm:ss"); - /** - * Skip a test if the FS isn't marked as supporting magic commits. - * @param fs filesystem - */ - public static void assumeMagicCommitEnabled(S3AFileSystem fs) - throws IOException { - assume("Magic commit option disabled on " + fs, - fs.hasPathCapability( - fs.getWorkingDirectory(), - CommitConstants.STORE_CAPABILITY_MAGIC_COMMITTER)); - } - /** * Probe for the configuration containing a specific credential provider. * If the list is empty, there will be no match, even if the named provider diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java index 21f268dfb2..08f4f8bc9d 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java @@ -22,6 +22,7 @@ import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.s3a.audit.AuditTestSupport; import org.apache.hadoop.fs.s3a.commit.PutTracker; +import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext; import org.apache.hadoop.util.Progressable; import org.junit.Before; @@ -65,7 +66,8 @@ private S3ABlockOutputStream.BlockOutputStreamBuilder mockS3ABuilder() { .withKey("") .withProgress(progressable) .withPutTracker(putTracker) - .withWriteOperations(oHelper); + .withWriteOperations(oHelper) + .withPutOptions(PutObjectOptions.keepingDirs()); return builder; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditAccessChecks.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditAccessChecks.java index a5a1b454d0..4b0b65fd78 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditAccessChecks.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditAccessChecks.java @@ -75,7 +75,9 @@ public Configuration createConfiguration() { @Override public void setup() throws Exception { super.setup(); - auditor = (AccessCheckingAuditor) getFileSystem().getAuditor(); + final S3AFileSystem fs = getFileSystem(); + auditor = (AccessCheckingAuditor) fs.getAuditor(); + setSpanSource(fs); } @Test diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java index 86bfa2bb07..51eac7e8cc 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java @@ -47,9 +47,10 @@ import org.apache.hadoop.fs.s3a.S3ATestConstants; import org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider; import org.apache.hadoop.fs.s3a.commit.CommitConstants; -import org.apache.hadoop.fs.s3a.commit.CommitOperations; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; +import org.apache.hadoop.fs.s3a.commit.impl.CommitContext; +import org.apache.hadoop.fs.s3a.commit.impl.CommitOperations; import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool; import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics; @@ -72,7 +73,7 @@ * Tests use of assumed roles. * Only run if an assumed role is provided. */ -@SuppressWarnings({"IOResourceOpenedButNotSafelyClosed", "ThrowableNotThrown"}) +@SuppressWarnings("ThrowableNotThrown") public class ITestAssumeRole extends AbstractS3ATestBase { private static final Logger LOG = @@ -563,9 +564,9 @@ public void testRestrictedCommitActions() throws Throwable { roleFS = (S3AFileSystem) writeableDir.getFileSystem(conf); CommitterStatistics committerStatistics = fs.newCommitterStatistics(); CommitOperations fullOperations = new CommitOperations(fs, - committerStatistics); + committerStatistics, "/"); CommitOperations operations = new CommitOperations(roleFS, - committerStatistics); + committerStatistics, "/"); File localSrc = File.createTempFile("source", ""); writeCSVData(localSrc); @@ -595,37 +596,37 @@ public void testRestrictedCommitActions() throws Throwable { SinglePendingCommit pending = fullOperations.uploadFileToPendingCommit(src, dest, "", uploadPartSize, progress); - pending.save(fs, new Path(readOnlyDir, - name + CommitConstants.PENDING_SUFFIX), true); + pending.save(fs, + new Path(readOnlyDir, name + CommitConstants.PENDING_SUFFIX), + SinglePendingCommit.serializer()); assertTrue(src.delete()); })); progress.assertCount("progress counter is not expected", range); - try { + try(CommitContext commitContext = + operations.createCommitContextForTesting(uploadDest, + null, 0)) { // we expect to be able to list all the files here Pair>> pendingCommits = operations.loadSinglePendingCommits(readOnlyDir, - true); + true, commitContext); // all those commits must fail List commits = pendingCommits.getLeft().getCommits(); assertEquals(range, commits.size()); - try(CommitOperations.CommitContext commitContext - = operations.initiateCommitOperation(uploadDest)) { - commits.parallelStream().forEach( - (c) -> { - CommitOperations.MaybeIOE maybeIOE = - commitContext.commit(c, "origin"); - Path path = c.destinationPath(); - assertCommitAccessDenied(path, maybeIOE); - }); - } + commits.parallelStream().forEach( + (c) -> { + CommitOperations.MaybeIOE maybeIOE = + commitContext.commit(c, "origin"); + Path path = c.destinationPath(); + assertCommitAccessDenied(path, maybeIOE); + }); // fail of all list and abort of .pending files. LOG.info("abortAllSinglePendingCommits({})", readOnlyDir); assertCommitAccessDenied(readOnlyDir, - operations.abortAllSinglePendingCommits(readOnlyDir, true)); + operations.abortAllSinglePendingCommits(readOnlyDir, commitContext, true)); // try writing a magic file Path magicDestPath = new Path(readOnlyDir, diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java index 17ce7e4ddc..9d9000cafb 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java @@ -18,9 +18,12 @@ package org.apache.hadoop.fs.s3a.commit; +import java.io.File; import java.io.IOException; +import java.time.LocalDateTime; +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatterBuilder; import java.util.List; -import java.util.stream.Collectors; import org.assertj.core.api.Assertions; import org.slf4j.Logger; @@ -34,17 +37,20 @@ import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; import org.apache.hadoop.fs.s3a.S3AFileSystem; -import org.apache.hadoop.fs.s3a.WriteOperationHelper; -import org.apache.hadoop.fs.store.audit.AuditSpan; import org.apache.hadoop.fs.s3a.commit.files.SuccessData; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.RecordWriter; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.TypeConverter; +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.task.TaskAttemptContextImpl; import org.apache.hadoop.mapreduce.v2.api.records.JobId; import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils; +import static java.time.temporal.ChronoField.DAY_OF_MONTH; +import static java.time.temporal.ChronoField.MONTH_OF_YEAR; +import static java.time.temporal.ChronoField.YEAR; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.MultipartTestUtils.listMultipartUploads; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; @@ -60,6 +66,17 @@ public abstract class AbstractCommitITest extends AbstractS3ATestBase { private static final Logger LOG = LoggerFactory.getLogger(AbstractCommitITest.class); + /** + * Helper class for commit operations and assertions. + */ + private CommitterTestHelper testHelper; + + /** + * Directory for job summary reports. + * This should be set up in test suites testing against real object stores. + */ + private File reportDir; + /** * Creates a configuration for commit operations: commit is enabled in the FS * and output is multipart to on-heap arrays. @@ -81,6 +98,8 @@ protected Configuration createConfiguration() { conf.setLong(MIN_MULTIPART_THRESHOLD, MULTIPART_MIN_SIZE); conf.setInt(MULTIPART_SIZE, MULTIPART_MIN_SIZE); conf.set(FAST_UPLOAD_BUFFER, FAST_UPLOAD_BUFFER_ARRAY); + // and bind the report dir + conf.set(OPT_SUMMARY_REPORT_DIR, reportDir.toURI().toString()); return conf; } @@ -92,6 +111,36 @@ public Logger log() { return LOG; } + /** + * Get directory for reports; valid after + * setup. + * @return where success/failure reports go. + */ + protected File getReportDir() { + return reportDir; + } + + @Override + public void setup() throws Exception { + // set the manifest committer to a localfs path for reports across + // all threads. + // do this before superclass setup so reportDir is non-null there + // and can be used in creating the configuration. + reportDir = new File(getProjectBuildDir(), "reports"); + reportDir.mkdirs(); + + super.setup(); + testHelper = new CommitterTestHelper(getFileSystem()); + } + + /** + * Get helper class. + * @return helper; only valid after setup. + */ + public CommitterTestHelper getTestHelper() { + return testHelper; + } + /*** * Bind to the named committer. * @@ -117,12 +166,14 @@ public void rmdir(Path dir, Configuration conf) throws IOException { if (dir != null) { describe("deleting %s", dir); FileSystem fs = dir.getFileSystem(conf); + fs.delete(dir, true); + } } /** - * Create a random Job ID using the fork ID as part of the number. + * Create a random Job ID using the fork ID and the current time. * @return fork ID string in a format parseable by Jobs * @throws Exception failure */ @@ -132,7 +183,14 @@ public static String randomJobId() throws Exception { String trailingDigits = testUniqueForkId.substring(l - 4, l); try { int digitValue = Integer.valueOf(trailingDigits); - return String.format("20070712%04d_%04d", + DateTimeFormatter formatter = new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .appendValue(YEAR, 4) + .appendValue(MONTH_OF_YEAR, 2) + .appendValue(DAY_OF_MONTH, 2) + .toFormatter(); + return String.format("%s%04d_%04d", + LocalDateTime.now().format(formatter), (long)(Math.random() * 1000), digitValue); } catch (NumberFormatException e) { @@ -146,22 +204,9 @@ public static String randomJobId() throws Exception { * @return a count of aborts * @throws IOException trouble. */ - protected int abortMultipartUploadsUnderPath(Path path) throws IOException { - S3AFileSystem fs = getFileSystem(); - if (fs != null && path != null) { - String key = fs.pathToKey(path); - int count = 0; - try (AuditSpan span = span()) { - WriteOperationHelper writeOps = fs.getWriteOperationHelper(); - count = writeOps.abortMultipartUploadsUnderPath(key); - if (count > 0) { - log().info("Multipart uploads deleted: {}", count); - } - } - return count; - } else { - return 0; - } + protected void abortMultipartUploadsUnderPath(Path path) throws IOException { + getTestHelper() + .abortMultipartUploadsUnderPath(path); } /** @@ -183,10 +228,9 @@ protected void assertMultipartUploadsPending(Path path) throws IOException { protected void assertNoMultipartUploadsPending(Path path) throws IOException { List uploads = listMultipartUploads(getFileSystem(), pathToPrefix(path)); - if (!uploads.isEmpty()) { - String result = uploads.stream().collect(Collectors.joining("\n")); - fail("Multipart uploads in progress under " + path + " \n" + result); - } + Assertions.assertThat(uploads) + .describedAs("Multipart uploads in progress under " + path) + .isEmpty(); } /** @@ -341,6 +385,13 @@ public static SuccessData validateSuccessFile(final Path outputPath, .describedAs("JobID in " + commitDetails) .isEqualTo(jobId); } + // also load as a manifest success data file + // to verify consistency and that the CLI tool works. + Path success = new Path(outputPath, _SUCCESS); + final ManifestPrinter showManifest = new ManifestPrinter(); + ManifestSuccessData manifestSuccessData = + showManifest.loadAndPrintManifest(fs, success); + return successData; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java index bf99d49576..b193cca03d 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java @@ -126,7 +126,7 @@ public abstract class AbstractITCommitProtocol extends AbstractCommitITest { private static final Text VAL_2 = new Text("val2"); /** A job to abort in test case teardown. */ - private List abortInTeardown = new ArrayList<>(1); + private final List abortInTeardown = new ArrayList<>(1); private final StandardCommitterFactory standardCommitterFactory = new StandardCommitterFactory(); @@ -562,7 +562,7 @@ protected void commit(AbstractS3ACommitter committer, describe("\ncommitting job"); committer.commitJob(jContext); describe("commit complete\n"); - verifyCommitterHasNoThreads(committer); + } } @@ -617,6 +617,9 @@ public void testRecoveryAndCleanup() throws Exception { assertNotNull("null outputPath in committer " + committer, committer.getOutputPath()); + // note the task attempt path. + Path job1TaskAttempt0Path = committer.getTaskAttemptPath(tContext); + // Commit the task. This will promote data and metadata to where // job commits will pick it up on commit or abort. commitTask(committer, tContext); @@ -636,6 +639,15 @@ public void testRecoveryAndCleanup() throws Exception { intercept(PathCommitException.class, "recover", () -> committer2.recoverTask(tContext2)); + // the new task attempt path is different from the first, because the + // job attempt counter is used in the path + final Path job2TaskAttempt0Path = committer2.getTaskAttemptPath(tContext2); + LOG.info("Job attempt 1 task attempt path {}; attempt 2 path {}", + job1TaskAttempt0Path, job2TaskAttempt0Path); + assertNotEquals("Task attempt paths must differ", + job1TaskAttempt0Path, + job2TaskAttempt0Path); + // at this point, task attempt 0 has failed to recover // it should be abortable though. This will be a no-op as it already // committed @@ -645,7 +657,7 @@ public void testRecoveryAndCleanup() throws Exception { committer2.abortJob(jContext2, JobStatus.State.KILLED); // now, state of system may still have pending data assertNoMultipartUploadsPending(outDir); - verifyCommitterHasNoThreads(committer2); + } protected void assertTaskAttemptPathDoesNotExist( @@ -747,7 +759,9 @@ private void validateMapFileOutputContent( assertPathExists("Map output", expectedMapDir); assertIsDirectory(expectedMapDir); FileStatus[] files = fs.listStatus(expectedMapDir); - assertTrue("No files found in " + expectedMapDir, files.length > 0); + Assertions.assertThat(files) + .describedAs("Files found in " + expectedMapDir) + .hasSizeGreaterThan(0); assertPathExists("index file in " + expectedMapDir, new Path(expectedMapDir, MapFile.INDEX_FILE_NAME)); assertPathExists("data file in " + expectedMapDir, @@ -795,9 +809,9 @@ public void testCommitLifecycle() throws Exception { try { applyLocatedFiles(getFileSystem().listFiles(outDir, false), - (status) -> - assertFalse("task committed file to dest :" + status, - status.getPath().toString().contains("part"))); + (status) -> Assertions.assertThat(status.getPath().toString()) + .describedAs("task committed file to dest :" + status) + .doesNotContain("part")); } catch (FileNotFoundException ignored) { log().info("Outdir {} is not created by task commit phase ", outDir); @@ -1071,27 +1085,34 @@ public void testMapFileOutputCommitter() throws Exception { // hidden filenames (_ or . prefixes) describe("listing"); FileStatus[] filtered = fs.listStatus(outDir, HIDDEN_FILE_FILTER); - assertEquals("listed children under " + ls, - 1, filtered.length); + Assertions.assertThat(filtered) + .describedAs("listed children under " + ls) + .hasSize(1); FileStatus fileStatus = filtered[0]; - assertTrue("Not the part file: " + fileStatus, - fileStatus.getPath().getName().startsWith(PART_00000)); + Assertions.assertThat(fileStatus.getPath().getName()) + .describedAs("Not a part file: " + fileStatus) + .startsWith(PART_00000); describe("getReaders()"); - assertEquals("Number of MapFile.Reader entries with shared FS " - + outDir + " : " + ls, - 1, getReaders(fs, outDir, conf).length); + Assertions.assertThat(getReaders(fs, outDir, conf)) + .describedAs("Number of MapFile.Reader entries with shared FS %s: %s", + outDir, ls) + .hasSize(1); describe("getReaders(new FS)"); FileSystem fs2 = FileSystem.get(outDir.toUri(), conf); - assertEquals("Number of MapFile.Reader entries with shared FS2 " - + outDir + " : " + ls, - 1, getReaders(fs2, outDir, conf).length); + Assertions.assertThat(getReaders(fs2, outDir, conf)) + .describedAs("Number of MapFile.Reader entries with shared FS2 %s: %s", + outDir, ls) + .hasSize(1); describe("MapFileOutputFormat.getReaders"); - assertEquals("Number of MapFile.Reader entries with new FS in " - + outDir + " : " + ls, - 1, MapFileOutputFormat.getReaders(outDir, conf).length); + + Assertions.assertThat(MapFileOutputFormat.getReaders(outDir, conf)) + .describedAs("Number of MapFile.Reader entries with new FS in %s: %s", + outDir, ls) + .hasSize(1); + } /** Open the output generated by this format. */ @@ -1165,7 +1186,7 @@ public void testAbortTaskThenJob() throws Exception { committer.abortJob(jobData.jContext, JobStatus.State.FAILED); assertJobAbortCleanedUp(jobData); - verifyCommitterHasNoThreads(committer); + } /** @@ -1219,7 +1240,7 @@ public void testFailAbort() throws Exception { // try again; expect abort to be idempotent. committer.abortJob(jContext, JobStatus.State.FAILED); assertNoMultipartUploadsPending(outDir); - verifyCommitterHasNoThreads(committer); + } public void assertPart0000DoesNotExist(Path dir) throws Exception { @@ -1433,7 +1454,7 @@ public void testAMWorkflow() throws Throwable { AbstractS3ACommitter committer2 = (AbstractS3ACommitter) outputFormat.getOutputCommitter(newAttempt); committer2.abortTask(tContext); - verifyCommitterHasNoThreads(committer2); + assertNoMultipartUploadsPending(getOutDir()); } @@ -1777,11 +1798,10 @@ public void testS3ACommitterFactoryBinding() throws Throwable { conf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID, 1); TaskAttemptContext tContext = new TaskAttemptContextImpl(conf, taskAttempt0); - String name = getCommitterName(); S3ACommitterFactory factory = new S3ACommitterFactory(); - assertEquals("Wrong committer from factory", - createCommitter(outDir, tContext).getClass(), - factory.createOutputCommitter(outDir, tContext).getClass()); + Assertions.assertThat(factory.createOutputCommitter(outDir, tContext).getClass()) + .describedAs("Committer from factory with name %s", getCommitterName()) + .isEqualTo(createCommitter(outDir, tContext).getClass()); } /** @@ -1830,7 +1850,7 @@ protected void validateTaskAttemptWorkingDirectory( protected void commitTask(final AbstractS3ACommitter committer, final TaskAttemptContext tContext) throws IOException { committer.commitTask(tContext); - verifyCommitterHasNoThreads(committer); + } /** @@ -1842,15 +1862,7 @@ protected void commitTask(final AbstractS3ACommitter committer, protected void commitJob(final AbstractS3ACommitter committer, final JobContext jContext) throws IOException { committer.commitJob(jContext); - verifyCommitterHasNoThreads(committer); + } - /** - * Verify that the committer does not have a thread pool. - * @param committer committer to validate. - */ - protected void verifyCommitterHasNoThreads(AbstractS3ACommitter committer) { - assertFalse("Committer has an active thread pool", - committer.hasThreadPool()); - } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/CommitterTestHelper.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/CommitterTestHelper.java new file mode 100644 index 0000000000..cd703f96da --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/CommitterTestHelper.java @@ -0,0 +1,148 @@ +/* + * 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.s3a.commit; + +import java.io.IOException; +import java.util.List; + +import org.assertj.core.api.Assertions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.MultipartTestUtils; +import org.apache.hadoop.fs.s3a.S3AFileSystem; + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyPathExists; +import static org.apache.hadoop.fs.s3a.commit.CommitConstants.BASE; +import static org.apache.hadoop.fs.s3a.commit.CommitConstants.MAGIC; +import static org.apache.hadoop.fs.s3a.commit.CommitConstants.STREAM_CAPABILITY_MAGIC_OUTPUT; +import static org.apache.hadoop.fs.s3a.commit.CommitConstants.XA_MAGIC_MARKER; +import static org.apache.hadoop.fs.s3a.commit.impl.CommitOperations.extractMagicFileLength; + +/** + * Helper for committer tests: extra assertions and the like. + */ +public class CommitterTestHelper { + + private static final Logger LOG = + LoggerFactory.getLogger(CommitterTestHelper.class); + + /** + * Filesystem under test. + */ + private final S3AFileSystem fileSystem; + + /** + * Constructor. + * @param fileSystem filesystem to work with. + */ + public CommitterTestHelper(S3AFileSystem fileSystem) { + this.fileSystem = requireNonNull(fileSystem); + } + + /** + * Get the filesystem. + * @return the filesystem. + */ + public S3AFileSystem getFileSystem() { + return fileSystem; + } + + /** + * Assert a path refers to a marker file of an expected length; + * the length is extracted from the custom header. + * @param path magic file. + * @param dataSize expected data size + * @throws IOException IO failure + */ + public void assertIsMarkerFile(Path path, long dataSize) throws IOException { + final S3AFileSystem fs = getFileSystem(); + FileStatus status = verifyPathExists(fs, + "uploaded file commit", path); + Assertions.assertThat(status.getLen()) + .describedAs("Marker File file %s: %s", path, status) + .isEqualTo(0); + Assertions.assertThat(extractMagicFileLength(fs, path)) + .describedAs("XAttribute " + XA_MAGIC_MARKER + " of " + path) + .isNotEmpty() + .hasValue(dataSize); + } + + /** + * Assert a file does not have the magic marker header. + * @param path magic file. + * @throws IOException IO failure + */ + public void assertFileLacksMarkerHeader(Path path) throws IOException { + Assertions.assertThat(extractMagicFileLength(getFileSystem(), + path)) + .describedAs("XAttribute " + XA_MAGIC_MARKER + " of " + path) + .isEmpty(); + } + + /** + * Create a new path which has the same filename as the dest file, but + * is in a magic directory under the destination dir. + * @param destFile final destination file + * @return magic path + */ + public static Path makeMagic(Path destFile) { + return new Path(destFile.getParent(), + MAGIC + '/' + BASE + "/" + destFile.getName()); + } + + /** + * Assert that an output stream is magic. + * @param stream stream to probe. + */ + public static void assertIsMagicStream(final FSDataOutputStream stream) { + Assertions.assertThat(stream.hasCapability(STREAM_CAPABILITY_MAGIC_OUTPUT)) + .describedAs("Stream capability %s in stream %s", + STREAM_CAPABILITY_MAGIC_OUTPUT, stream) + .isTrue(); + } + + /** + * Abort all multipart uploads under a path. + * @param path path for uploads to abort; may be null + * @return a count of aborts + * @throws IOException trouble. + */ + public void abortMultipartUploadsUnderPath(Path path) { + + MultipartTestUtils.clearAnyUploads(getFileSystem(), path); + } + + /** + * Get a list of all pending uploads under a prefix, one which can be printed. + * @param prefix prefix to look under + * @return possibly empty list + * @throws IOException IO failure. + */ + public List listMultipartUploads( + String prefix) throws IOException { + + return MultipartTestUtils.listMultipartUploads(getFileSystem(), prefix); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperationCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperationCost.java new file mode 100644 index 0000000000..d0c86b738c --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperationCost.java @@ -0,0 +1,325 @@ +/* + * 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.s3a.commit; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; + +import org.assertj.core.api.Assertions; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.commit.files.PersistentCommitData; +import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; +import org.apache.hadoop.fs.s3a.commit.impl.CommitOperations; +import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest; +import org.apache.hadoop.fs.statistics.IOStatisticsLogging; + +import static org.apache.hadoop.fs.s3a.Statistic.ACTION_HTTP_GET_REQUEST; +import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_MAGIC_FILES_CREATED; +import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_MAGIC_MARKER_PUT; +import static org.apache.hadoop.fs.s3a.Statistic.DIRECTORIES_CREATED; +import static org.apache.hadoop.fs.s3a.Statistic.FAKE_DIRECTORIES_DELETED; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_BULK_DELETE_REQUEST; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_DELETE_REQUEST; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_LIST_REQUEST; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_METADATA_REQUESTS; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_MULTIPART_UPLOAD_INITIATED; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_PUT_REQUESTS; +import static org.apache.hadoop.fs.s3a.commit.CommitConstants.MAGIC; +import static org.apache.hadoop.fs.s3a.commit.CommitterTestHelper.assertIsMagicStream; +import static org.apache.hadoop.fs.s3a.commit.CommitterTestHelper.makeMagic; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.LIST_FILES_LIST_OP; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.LIST_OPERATION; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.NO_HEAD_OR_LIST; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString; +import static org.apache.hadoop.util.functional.RemoteIterators.toList; + +/** + * Assert cost of commit operations; + *
      + *
    1. Even on marker deleting filesystems, + * operations under magic dirs do not trigger marker deletion.
    2. + *
    3. Loading pending files from FileStatus entries skips HEAD checks.
    4. + *
    5. Mkdir under magic dirs doesn't check ancestor or dest type
    6. + *
    + */ +public class ITestCommitOperationCost extends AbstractS3ACostTest { + + private static final Logger LOG = + LoggerFactory.getLogger(ITestCommitOperationCost.class); + + /** + * Helper for the tests. + */ + private CommitterTestHelper testHelper; + + /** + * Create with markers kept, always. + */ + public ITestCommitOperationCost() { + super(false); + } + + @Override + public void setup() throws Exception { + super.setup(); + testHelper = new CommitterTestHelper(getFileSystem()); + } + + @Override + public void teardown() throws Exception { + try { + if (testHelper != null) { + testHelper.abortMultipartUploadsUnderPath(methodPath()); + } + } finally { + super.teardown(); + } + } + + /** + * Get a method-relative path. + * @param filename filename + * @return new path + * @throws IOException failure to create/parse the path. + */ + private Path methodSubPath(String filename) throws IOException { + return new Path(methodPath(), filename); + } + + /** + * Return the FS IOStats, prettified. + * @return string for assertions. + */ + protected String fileSystemIOStats() { + return ioStatisticsToPrettyString(getFileSystem().getIOStatistics()); + } + + @Test + public void testMagicMkdir() throws Throwable { + describe("Mkdirs __magic always skips dir marker deletion"); + S3AFileSystem fs = getFileSystem(); + Path baseDir = methodPath(); + // create dest dir marker, always + fs.mkdirs(baseDir); + Path magicDir = new Path(baseDir, MAGIC); + verifyMetrics(() -> { + fs.mkdirs(magicDir); + return fileSystemIOStats(); + }, + with(OBJECT_BULK_DELETE_REQUEST, 0), + with(OBJECT_DELETE_REQUEST, 0), + with(DIRECTORIES_CREATED, 1)); + verifyMetrics(() -> { + fs.delete(magicDir, true); + return fileSystemIOStats(); + }, + with(OBJECT_BULK_DELETE_REQUEST, 0), + with(OBJECT_DELETE_REQUEST, 1), + with(DIRECTORIES_CREATED, 0)); + assertPathExists("parent", baseDir); + } + + /** + * When a magic subdir is deleted, parent dirs are not recreated. + */ + @Test + public void testMagicMkdirs() throws Throwable { + describe("Mkdirs __magic/subdir always skips dir marker deletion"); + S3AFileSystem fs = getFileSystem(); + Path baseDir = methodPath(); + Path magicDir = new Path(baseDir, MAGIC); + fs.delete(baseDir, true); + + Path magicSubdir = new Path(magicDir, "subdir"); + verifyMetrics(() -> { + fs.mkdirs(magicSubdir, FsPermission.getDirDefault()); + return "after mkdirs " + fileSystemIOStats(); + }, + always(LIST_OPERATION), + with(OBJECT_BULK_DELETE_REQUEST, 0), + with(OBJECT_DELETE_REQUEST, 0), + with(DIRECTORIES_CREATED, 1)); + assertPathExists("magicSubdir", magicSubdir); + + verifyMetrics(() -> { + fs.delete(magicSubdir, true); + return "after delete " + fileSystemIOStats(); + }, + with(OBJECT_BULK_DELETE_REQUEST, 0), + with(OBJECT_DELETE_REQUEST, 1), + with(OBJECT_LIST_REQUEST, 1), + with(OBJECT_METADATA_REQUESTS, 1), + with(DIRECTORIES_CREATED, 0)); + // no marker dir creation + assertPathDoesNotExist("magicDir", magicDir); + assertPathDoesNotExist("baseDir", baseDir); + } + + /** + * Active stream; a field is used so closures can write to + * it. + */ + private FSDataOutputStream stream; + + /** + * Abort any active stream. + * @throws IOException failure + */ + private void abortActiveStream() throws IOException { + if (stream != null) { + stream.abort(); + stream.close(); + } + } + + @Test + public void testCostOfCreatingMagicFile() throws Throwable { + describe("Files created under magic paths skip existence checks and marker deletes"); + S3AFileSystem fs = getFileSystem(); + Path destFile = methodSubPath("file.txt"); + fs.delete(destFile.getParent(), true); + Path magicDest = makeMagic(destFile); + + // when the file is created, there is no check for overwrites + // or the dest being a directory, even if overwrite=false + try { + verifyMetrics(() -> { + stream = fs.create(magicDest, false); + return stream.toString(); + }, + always(NO_HEAD_OR_LIST), + with(COMMITTER_MAGIC_FILES_CREATED, 1), + with(COMMITTER_MAGIC_MARKER_PUT, 0), + with(OBJECT_MULTIPART_UPLOAD_INITIATED, 1)); + assertIsMagicStream(stream); + + stream.write("hello".getBytes(StandardCharsets.UTF_8)); + + // when closing, there will be no directories deleted + // we do expect two PUT requests, because the marker and manifests + // are both written + LOG.info("closing magic stream to {}", magicDest); + verifyMetrics(() -> { + stream.close(); + return stream.toString(); + }, + always(NO_HEAD_OR_LIST), + with(OBJECT_PUT_REQUESTS, 2), + with(COMMITTER_MAGIC_MARKER_PUT, 2), + with(OBJECT_BULK_DELETE_REQUEST, 0), + with(OBJECT_DELETE_REQUEST, 0)); + + } catch (Exception e) { + abortActiveStream(); + throw e; + } + // list the manifests + final CommitOperations commitOperations = new CommitOperations(fs); + List pending = verifyMetrics(() -> + toList(commitOperations. + locateAllSinglePendingCommits(magicDest.getParent(), false)), + always(LIST_FILES_LIST_OP)); + Assertions.assertThat(pending) + .describedAs("pending commits") + .hasSize(1); + + // load the only pending commit + SinglePendingCommit singleCommit = verifyMetrics(() -> + PersistentCommitData.load(fs, + pending.get(0), + SinglePendingCommit.serializer()), + always(NO_HEAD_OR_LIST), + with(ACTION_HTTP_GET_REQUEST, 1)); + + // commit it through the commit operations. + verifyMetrics(() -> { + commitOperations.commitOrFail(singleCommit); + return ioStatisticsToPrettyString( + commitOperations.getIOStatistics()); + }, + always(NO_HEAD_OR_LIST), // no probes for the dest path + with(FAKE_DIRECTORIES_DELETED, 0), // no fake dirs + with(OBJECT_DELETE_REQUEST, 0)); // no deletes + + LOG.info("Final Statistics {}", + IOStatisticsLogging.ioStatisticsToPrettyString(stream.getIOStatistics())); + } + + /** + * saving pending files MUST NOT trigger HEAD/LIST calls + * when created under a magic path; when opening + * with an S3AFileStatus the HEAD will be skipped too. + */ + @Test + public void testCostOfSavingLoadingPendingFile() throws Throwable { + describe("Verify costs of saving .pending file under a magic path"); + + S3AFileSystem fs = getFileSystem(); + Path partDir = methodSubPath("file.pending"); + Path destFile = new Path(partDir, "file.pending"); + Path magicDest = makeMagic(destFile); + // create a pending file with minimal values needed + // for validation to work + final SinglePendingCommit commit = new SinglePendingCommit(); + commit.touch(System.currentTimeMillis()); + commit.setUri(destFile.toUri().toString()); + commit.setBucket(fs.getBucket()); + commit.setLength(0); + commit.setDestinationKey(fs.pathToKey(destFile)); + commit.setUploadId("uploadId"); + commit.setEtags(new ArrayList<>()); + // fail fast if the commit data is incomplete + commit.validate(); + + // save the file: no checks will be made + verifyMetrics(() -> { + commit.save(fs, magicDest, + SinglePendingCommit.serializer()); + return commit.toString(); + }, + with(COMMITTER_MAGIC_FILES_CREATED, 0), + always(NO_HEAD_OR_LIST), + with(OBJECT_BULK_DELETE_REQUEST, 0), + with(OBJECT_DELETE_REQUEST, 0) + ); + + LOG.info("File written; Validating"); + testHelper.assertFileLacksMarkerHeader(magicDest); + FileStatus status = fs.getFileStatus(magicDest); + + LOG.info("Reading file {}", status); + // opening a file with a status passed in will skip the HEAD + verifyMetrics(() -> + PersistentCommitData.load(fs, status, SinglePendingCommit.serializer()), + always(NO_HEAD_OR_LIST), + with(ACTION_HTTP_GET_REQUEST, 1)); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperations.java index 3f0e2e7a13..a76a65be8b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperations.java @@ -23,8 +23,8 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.UUID; -import com.amazonaws.services.s3.model.PartETag; import org.assertj.core.api.Assertions; import org.junit.Test; import org.slf4j.Logger; @@ -33,12 +33,15 @@ import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FSDataOutputStreamBuilder; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.auth.ProgressCounter; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; +import org.apache.hadoop.fs.s3a.commit.impl.CommitContext; +import org.apache.hadoop.fs.s3a.commit.impl.CommitOperations; import org.apache.hadoop.fs.s3a.commit.magic.MagicCommitTracker; import org.apache.hadoop.fs.s3a.commit.magic.MagicS3GuardCommitter; import org.apache.hadoop.fs.s3a.commit.magic.MagicS3GuardCommitterFactory; @@ -52,11 +55,13 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; -import static org.apache.hadoop.fs.s3a.commit.CommitOperations.extractMagicFileLength; import static org.apache.hadoop.fs.s3a.commit.CommitUtils.*; +import static org.apache.hadoop.fs.s3a.commit.CommitterTestHelper.assertIsMagicStream; import static org.apache.hadoop.fs.s3a.commit.MagicCommitPaths.*; import static org.apache.hadoop.fs.s3a.Constants.*; +import static org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext.EMPTY_BLOCK_OUTPUT_STREAM_STATISTICS; import static org.apache.hadoop.mapreduce.lib.output.PathOutputCommitterFactory.*; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Test the low-level binding of the S3A FS to the magic commit mechanism, @@ -69,6 +74,8 @@ public class ITestCommitOperations extends AbstractCommitITest { private static final byte[] DATASET = dataset(1000, 'a', 32); private static final String S3A_FACTORY_KEY = String.format( COMMITTER_FACTORY_SCHEME_PATTERN, "s3a"); + private static final String JOB_ID = UUID.randomUUID().toString(); + private ProgressCounter progress; @Override @@ -94,11 +101,13 @@ public void testCreateTrackerNormalPath() throws Throwable { MagicCommitIntegration integration = new MagicCommitIntegration(fs, true); String filename = "notdelayed.txt"; - Path destFile = methodPath(filename); + Path destFile = methodSubPath(filename); String origKey = fs.pathToKey(destFile); - PutTracker tracker = integration.createTracker(destFile, origKey); - assertFalse("wrong type: " + tracker + " for " + destFile, - tracker instanceof MagicCommitTracker); + PutTracker tracker = integration.createTracker(destFile, origKey, + EMPTY_BLOCK_OUTPUT_STREAM_STATISTICS); + Assertions.assertThat(tracker) + .describedAs("Tracker for %s", destFile) + .isNotInstanceOf(MagicCommitTracker.class); } /** @@ -111,36 +120,45 @@ public void testCreateTrackerMagicPath() throws Throwable { MagicCommitIntegration integration = new MagicCommitIntegration(fs, true); String filename = "delayed.txt"; - Path destFile = methodPath(filename); + Path destFile = methodSubPath(filename); String origKey = fs.pathToKey(destFile); Path pendingPath = makeMagic(destFile); verifyIsMagicCommitPath(fs, pendingPath); String pendingPathKey = fs.pathToKey(pendingPath); - assertTrue("wrong path of " + pendingPathKey, - pendingPathKey.endsWith(filename)); + Assertions.assertThat(pendingPathKey) + .describedAs("pending path") + .endsWith(filename); final List elements = splitPathToElements(pendingPath); - assertEquals("splitPathToElements()", filename, lastElement(elements)); + Assertions.assertThat(lastElement(elements)) + .describedAs("splitPathToElements(%s)", pendingPath) + .isEqualTo(filename); List finalDestination = finalDestination(elements); - assertEquals("finalDestination()", - filename, - lastElement(finalDestination)); - final String destKey = elementsToKey(finalDestination); - assertEquals("destination key", origKey, destKey); + Assertions.assertThat(lastElement(finalDestination)) + .describedAs("finalDestination(%s)", pendingPath) + .isEqualTo(filename); + Assertions.assertThat(elementsToKey(finalDestination)) + .describedAs("destination key") + .isEqualTo(origKey); PutTracker tracker = integration.createTracker(pendingPath, - pendingPathKey); - assertTrue("wrong type: " + tracker + " for " + pendingPathKey, - tracker instanceof MagicCommitTracker); - assertEquals("tracker destination key", origKey, tracker.getDestKey()); + pendingPathKey, EMPTY_BLOCK_OUTPUT_STREAM_STATISTICS); + Assertions.assertThat(tracker) + .describedAs("Tracker for %s", pendingPathKey) + .isInstanceOf(MagicCommitTracker.class); + Assertions.assertThat(tracker.getDestKey()) + .describedAs("tracker destination key") + .isEqualTo(origKey); - Path pendingSuffixedPath = new Path(pendingPath, - "part-0000" + PENDING_SUFFIX); - assertFalse("still a delayed complete path " + pendingSuffixedPath, - fs.isMagicCommitPath(pendingSuffixedPath)); - Path pendingSet = new Path(pendingPath, - "part-0000" + PENDINGSET_SUFFIX); - assertFalse("still a delayed complete path " + pendingSet, - fs.isMagicCommitPath(pendingSet)); + assertNotDelayedWrite(new Path(pendingPath, + "part-0000" + PENDING_SUFFIX)); + assertNotDelayedWrite(new Path(pendingPath, + "part-0000" + PENDINGSET_SUFFIX)); + } + + private void assertNotDelayedWrite(Path pendingSuffixedPath) { + Assertions.assertThat(getFileSystem().isMagicCommitPath(pendingSuffixedPath)) + .describedAs("Expected %s to not be magic/delayed write", pendingSuffixedPath) + .isFalse(); } @Test @@ -148,7 +166,7 @@ public void testCreateAbortEmptyFile() throws Throwable { describe("create then abort an empty file; throttled"); S3AFileSystem fs = getFileSystem(); String filename = "empty-abort.txt"; - Path destFile = methodPath(filename); + Path destFile = methodSubPath(filename); Path pendingFilePath = makeMagic(destFile); touch(fs, pendingFilePath); @@ -160,13 +178,22 @@ public void testCreateAbortEmptyFile() throws Throwable { // abort,; rethrow on failure LOG.info("Abort call"); - actions.abortAllSinglePendingCommits(pendingDataPath.getParent(), true) - .maybeRethrow(); + Path parent = pendingDataPath.getParent(); + try (CommitContext commitContext = + actions.createCommitContextForTesting(parent, JOB_ID, 0)) { + actions.abortAllSinglePendingCommits(parent, commitContext, true) + .maybeRethrow(); + } assertPathDoesNotExist("pending file not deleted", pendingDataPath); assertPathDoesNotExist("dest file was created", destFile); } + /** + * Create a new commit operations instance for the test FS. + * @return commit operations. + * @throws IOException IO failure. + */ private CommitOperations newCommitOperations() throws IOException { return new CommitOperations(getFileSystem()); @@ -198,10 +225,16 @@ public void testCommitSmallFile() throws Throwable { @Test public void testAbortNonexistentDir() throws Throwable { describe("Attempt to abort a directory that does not exist"); - Path destFile = methodPath("testAbortNonexistentPath"); - newCommitOperations() - .abortAllSinglePendingCommits(destFile, true) - .maybeRethrow(); + Path destFile = methodSubPath("testAbortNonexistentPath"); + final CommitOperations operations = newCommitOperations(); + try (CommitContext commitContext + = operations.createCommitContextForTesting(destFile, JOB_ID, 0)) { + final CommitOperations.MaybeIOE outcome = operations + .abortAllSinglePendingCommits(destFile, commitContext, true); + outcome.maybeRethrow(); + Assertions.assertThat(outcome) + .isEqualTo(CommitOperations.MaybeIOE.NONE); + } } @Test @@ -244,7 +277,7 @@ public void testBaseRelativePath() throws Throwable { describe("Test creating file with a __base marker and verify that it ends" + " up in where expected"); S3AFileSystem fs = getFileSystem(); - Path destDir = methodPath("testBaseRelativePath"); + Path destDir = methodSubPath("testBaseRelativePath"); fs.delete(destDir, true); Path pendingBaseDir = new Path(destDir, MAGIC + "/child/" + BASE); String child = "subdir/child.txt"; @@ -270,14 +303,17 @@ public void testMarkerFileRename() fs.delete(destDir, true); Path magicDest = makeMagic(destFile); Path magicDir = magicDest.getParent(); - fs.mkdirs(magicDir); + fs.mkdirs(magicDest); // use the builder API to verify it works exactly the // same. - try (FSDataOutputStream stream = fs.createFile(magicDest) - .overwrite(true) - .recursive() - .build()) { + FSDataOutputStreamBuilder builder = fs.createFile(magicDest) + .overwrite(true); + builder.recursive(); + // this has a broken return type; not sure why + builder.must(FS_S3A_CREATE_PERFORMANCE, true); + + try (FSDataOutputStream stream = builder.build()) { assertIsMagicStream(stream); stream.write(DATASET); } @@ -286,9 +322,7 @@ public void testMarkerFileRename() fs.rename(magicDest, magic2); // the renamed file has no header - Assertions.assertThat(extractMagicFileLength(fs, magic2)) - .describedAs("XAttribute " + XA_MAGIC_MARKER + " of " + magic2) - .isEmpty(); + getTestHelper().assertFileLacksMarkerHeader(magic2); // abort the upload, which is driven by the .pending files // there must be 1 deleted file; during test debugging with aborted // runs there may be more. @@ -298,17 +332,6 @@ public void testMarkerFileRename() .isGreaterThanOrEqualTo(1); } - /** - * Assert that an output stream is magic. - * @param stream stream to probe. - */ - protected void assertIsMagicStream(final FSDataOutputStream stream) { - Assertions.assertThat(stream.hasCapability(STREAM_CAPABILITY_MAGIC_OUTPUT)) - .describedAs("Stream capability %s in stream %s", - STREAM_CAPABILITY_MAGIC_OUTPUT, stream) - .isTrue(); - } - /** * Create a file through the magic commit mechanism. * @param filename file to create (with __magic path.) @@ -318,39 +341,27 @@ protected void assertIsMagicStream(final FSDataOutputStream stream) { private void createCommitAndVerify(String filename, byte[] data) throws Exception { S3AFileSystem fs = getFileSystem(); - Path destFile = methodPath(filename); + Path destFile = methodSubPath(filename); fs.delete(destFile.getParent(), true); Path magicDest = makeMagic(destFile); assertPathDoesNotExist("Magic file should not exist", magicDest); long dataSize = data != null ? data.length : 0; - try(FSDataOutputStream stream = fs.create(magicDest, true)) { + try (FSDataOutputStream stream = fs.create(magicDest, true)) { assertIsMagicStream(stream); if (dataSize > 0) { stream.write(data); } - stream.close(); } - FileStatus status = fs.getFileStatus(magicDest); - assertEquals("Magic marker file is not zero bytes: " + status, - 0, 0); - Assertions.assertThat(extractMagicFileLength(fs, - magicDest)) - .describedAs("XAttribute " + XA_MAGIC_MARKER + " of " + magicDest) - .isNotEmpty() - .hasValue(dataSize); + getTestHelper().assertIsMarkerFile(magicDest, dataSize); commit(filename, destFile); verifyFileContents(fs, destFile, data); // the destination file doesn't have the attribute - Assertions.assertThat(extractMagicFileLength(fs, - destFile)) - .describedAs("XAttribute " + XA_MAGIC_MARKER + " of " + destFile) - .isEmpty(); + getTestHelper().assertFileLacksMarkerHeader(destFile); } /** * Commit the file, with before and after checks on the dest and magic * values. - * Failures can be set; they'll be reset after the commit. * @param filename filename of file * @param destFile destination path of file * @throws Exception any failure of the operation @@ -371,20 +382,21 @@ private void commit(String filename, Path destFile) throws IOException { + final CommitOperations actions = newCommitOperations(); validateIntermediateAndFinalPaths(magicFile, destFile); SinglePendingCommit commit = SinglePendingCommit.load(getFileSystem(), - validatePendingCommitData(filename, magicFile)); + validatePendingCommitData(filename, magicFile), + null, + SinglePendingCommit.serializer()); - commitOrFail(destFile, commit, newCommitOperations()); - - verifyCommitExists(commit); + commitOrFail(destFile, commit, actions); } private void commitOrFail(final Path destFile, final SinglePendingCommit commit, final CommitOperations actions) throws IOException { - try (CommitOperations.CommitContext commitContext - = actions.initiateCommitOperation(destFile)) { + try (CommitContext commitContext + = actions.createCommitContextForTesting(destFile, JOB_ID, 0)) { commitContext.commitOrFail(commit); } } @@ -401,26 +413,6 @@ private void validateIntermediateAndFinalPaths(Path magicFilePath, assertPathDoesNotExist("dest file was created", destFile); } - /** - * Verify that the path at the end of a commit exists. - * This does not validate the size. - * @param commit commit to verify - * @throws FileNotFoundException dest doesn't exist - * @throws ValidationFailure commit arg is invalid - * @throws IOException invalid commit, IO failure - */ - private void verifyCommitExists(SinglePendingCommit commit) - throws FileNotFoundException, ValidationFailure, IOException { - commit.validate(); - // this will force an existence check - Path path = getFileSystem().keyToQualifiedPath(commit.getDestinationKey()); - FileStatus status = getFileSystem().getFileStatus(path); - LOG.debug("Destination entry: {}", status); - if (!status.isFile()) { - throw new PathCommitException(path, "Not a file: " + status); - } - } - /** * Validate that a pending commit data file exists, load it and validate * its contents. @@ -443,14 +435,19 @@ private Path validatePendingCommitData(String filename, SinglePendingCommit persisted = SinglePendingCommit.serializer() .load(fs, pendingDataPath); persisted.validate(); - assertTrue("created timestamp wrong in " + persisted, - persisted.getCreated() > 0); - assertTrue("saved timestamp wrong in " + persisted, - persisted.getSaved() > 0); + Assertions.assertThat(persisted.getCreated()) + .describedAs("Created timestamp in %s", persisted) + .isGreaterThan(0); + Assertions.assertThat(persisted.getSaved()) + .describedAs("saved timestamp in %s", persisted) + .isGreaterThan(0); List etags = persisted.getEtags(); - assertEquals("etag list " + persisted, 1, etags.size()); - List partList = CommitOperations.toPartEtags(etags); - assertEquals("part list " + persisted, 1, partList.size()); + Assertions.assertThat(etags) + .describedAs("Etag list") + .hasSize(1); + Assertions.assertThat(CommitOperations.toPartEtags(etags)) + .describedAs("Etags to parts") + .hasSize(1); return pendingDataPath; } @@ -460,24 +457,16 @@ private Path validatePendingCommitData(String filename, * @return new path * @throws IOException failure to create/parse the path. */ - private Path methodPath(String filename) throws IOException { + private Path methodSubPath(String filename) throws IOException { return new Path(methodPath(), filename); } - /** - * Get a unique path for a method. - * @return a path - * @throws IOException - */ - protected Path methodPath() throws IOException { - return path(getMethodName()); - } - @Test public void testUploadEmptyFile() throws Throwable { + describe("Upload a zero byte file to a magic path"); File tempFile = File.createTempFile("commit", ".txt"); CommitOperations actions = newCommitOperations(); - Path dest = methodPath("testUploadEmptyFile"); + Path dest = methodSubPath("testUploadEmptyFile"); S3AFileSystem fs = getFileSystem(); fs.delete(dest, false); @@ -492,11 +481,14 @@ public void testUploadEmptyFile() throws Throwable { commitOrFail(dest, pendingCommit, actions); - FileStatus status = verifyPathExists(fs, - "uploaded file commit", dest); progress.assertCount("Progress counter should be 1.", 1); - assertEquals("File length in " + status, 0, status.getLen()); + FileStatus status = verifyPathExists(fs, + "uploaded file commit", dest); + Assertions.assertThat(status.getLen()) + .describedAs("Committed File file %s: %s", dest, status) + .isEqualTo(0); + getTestHelper().assertFileLacksMarkerHeader(dest); } @Test @@ -505,7 +497,7 @@ public void testUploadSmallFile() throws Throwable { String text = "hello, world"; FileUtils.write(tempFile, text, "UTF-8"); CommitOperations actions = newCommitOperations(); - Path dest = methodPath("testUploadSmallFile"); + Path dest = methodSubPath("testUploadSmallFile"); S3AFileSystem fs = getFileSystem(); fs.delete(dest, true); @@ -523,51 +515,57 @@ public void testUploadSmallFile() throws Throwable { commitOrFail(dest, pendingCommit, actions); String s = readUTF8(fs, dest, -1); - assertEquals(text, s); + Assertions.assertThat(s) + .describedAs("contents of committed file %s", dest) + .isEqualTo(text); progress.assertCount("Progress counter should be 1.", 1); } - @Test(expected = FileNotFoundException.class) + @Test public void testUploadMissingFile() throws Throwable { File tempFile = File.createTempFile("commit", ".txt"); tempFile.delete(); CommitOperations actions = newCommitOperations(); - Path dest = methodPath("testUploadMissingile"); - - actions.uploadFileToPendingCommit(tempFile, dest, null, - DEFAULT_MULTIPART_SIZE, progress); - progress.assertCount("Progress counter should be 1.", - 1); + Path dest = methodSubPath("testUploadMissingFile"); + intercept(FileNotFoundException.class, () -> + actions.uploadFileToPendingCommit(tempFile, dest, null, + DEFAULT_MULTIPART_SIZE, progress)); + progress.assertCount("Progress counter should be 0.", + 0); } @Test public void testRevertCommit() throws Throwable { - Path destFile = methodPath("part-0000"); + describe("Revert a commit; the destination file will be deleted"); + Path destFile = methodSubPath("part-0000"); S3AFileSystem fs = getFileSystem(); touch(fs, destFile); - CommitOperations actions = newCommitOperations(); SinglePendingCommit commit = new SinglePendingCommit(); + CommitOperations actions = newCommitOperations(); commit.setDestinationKey(fs.pathToKey(destFile)); - - actions.revertCommit(commit); - - assertPathExists("parent of reverted commit", destFile.getParent()); + newCommitOperations().revertCommit(commit); + assertPathDoesNotExist("should have been reverted", destFile); } @Test public void testRevertMissingCommit() throws Throwable { - Path destFile = methodPath("part-0000"); + Path destFile = methodSubPath("part-0000"); S3AFileSystem fs = getFileSystem(); fs.delete(destFile, false); - CommitOperations actions = newCommitOperations(); SinglePendingCommit commit = new SinglePendingCommit(); commit.setDestinationKey(fs.pathToKey(destFile)); + newCommitOperations().revertCommit(commit); + assertPathDoesNotExist("should have been reverted", destFile); + } - actions.revertCommit(commit); - - assertPathExists("parent of reverted (nonexistent) commit", - destFile.getParent()); + @Test + public void testFailuresInAbortListing() throws Throwable { + Path path = path("testFailuresInAbort"); + getFileSystem().mkdirs(path); + LOG.info("Aborting"); + newCommitOperations().abortPendingUploadsUnderPath(path); + LOG.info("Abort completed"); } /** @@ -578,16 +576,16 @@ public void testRevertMissingCommit() throws Throwable { @Test public void testWriteNormalStream() throws Throwable { S3AFileSystem fs = getFileSystem(); - assumeMagicCommitEnabled(fs); Path destFile = path("normal"); try (FSDataOutputStream out = fs.create(destFile, true)) { out.writeChars("data"); assertFalse("stream has magic output: " + out, out.hasCapability(STREAM_CAPABILITY_MAGIC_OUTPUT)); - out.close(); } FileStatus status = fs.getFileStatus(destFile); - assertTrue("Empty marker file: " + status, status.getLen() > 0); + Assertions.assertThat(status.getLen()) + .describedAs("Normal file %s: %s", destFile, status) + .isGreaterThan(0); } /** @@ -598,7 +596,7 @@ public void testBulkCommitFiles() throws Throwable { describe("verify bulk commit"); File localFile = File.createTempFile("commit", ".txt"); CommitOperations actions = newCommitOperations(); - Path destDir = methodPath("out"); + Path destDir = methodSubPath("out"); S3AFileSystem fs = getFileSystem(); fs.delete(destDir, false); @@ -612,7 +610,7 @@ public void testBulkCommitFiles() throws Throwable { destFile3); List commits = new ArrayList<>(3); - for (Path destination : destinations) { + for (Path destination: destinations) { SinglePendingCommit commit1 = actions.uploadFileToPendingCommit(localFile, destination, null, @@ -624,8 +622,8 @@ public void testBulkCommitFiles() throws Throwable { assertPathDoesNotExist("destination dir", destDir); assertPathDoesNotExist("subdirectory", subdir); LOG.info("Initiating commit operations"); - try (CommitOperations.CommitContext commitContext - = actions.initiateCommitOperation(destDir)) { + try (CommitContext commitContext + = actions.createCommitContextForTesting(destDir, JOB_ID, 0)) { LOG.info("Commit #1"); commitContext.commitOrFail(commits.get(0)); final String firstCommitContextString = commitContext.toString(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestS3ACommitterFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestS3ACommitterFactory.java index a8547d6728..2ad2568d5c 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestS3ACommitterFactory.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestS3ACommitterFactory.java @@ -157,7 +157,7 @@ public void testBindingsInFSConfig() throws Throwable { } /** - * Create an invalid committer via the FS binding, + * Create an invalid committer via the FS binding. */ public void testInvalidFileBinding() throws Throwable { taskConfRef.unset(FS_S3A_COMMITTER_NAME); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/TestTasks.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/TestTasks.java deleted file mode 100644 index 7ff5c3d280..0000000000 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/TestTasks.java +++ /dev/null @@ -1,569 +0,0 @@ -/* - * 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.s3a.commit; - -import java.io.IOException; -import java.util.Arrays; -import java.util.Collection; -import java.util.List; -import java.util.Optional; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.IntStream; - -import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hadoop.test.HadoopTestBase; - -import static org.apache.hadoop.test.LambdaTestUtils.intercept; - -/** - * Test Tasks class. - */ -@RunWith(Parameterized.class) -public class TestTasks extends HadoopTestBase { - private static final Logger LOG = LoggerFactory.getLogger(TestTasks.class); - public static final int ITEM_COUNT = 16; - private static final int FAILPOINT = 8; - - private final int numThreads; - /** - * Thread pool for task execution. - */ - private ExecutorService threadPool; - - /** - * Task submitter bonded to the thread pool, or - * null for the 0-thread case. - */ - Tasks.Submitter submitter; - private final CounterTask failingTask - = new CounterTask("failing committer", FAILPOINT, Item::commit); - - private final FailureCounter failures - = new FailureCounter("failures", 0, null); - private final CounterTask reverter - = new CounterTask("reverter", 0, Item::revert); - private final CounterTask aborter - = new CounterTask("aborter", 0, Item::abort); - - /** - * Test array for parameterized test runs: how many threads and - * to use. Threading makes some of the assertions brittle; there are - * more checks on single thread than parallel ops. - * @return a list of parameter tuples. - */ - @Parameterized.Parameters(name = "threads={0}") - public static Collection params() { - return Arrays.asList(new Object[][]{ - {0}, - {1}, - {3}, - {8}, - {16}, - }); - } - - - private List items; - - - /** - * Construct the parameterized test. - * @param numThreads number of threads - */ - public TestTasks(int numThreads) { - this.numThreads = numThreads; - } - - /** - * In a parallel test run there is more than one thread doing the execution. - * @return true if the threadpool size is >1 - */ - public boolean isParallel() { - return numThreads > 1; - } - - @Before - public void setup() { - items = IntStream.rangeClosed(1, ITEM_COUNT) - .mapToObj(i -> new Item(i, - String.format("With %d threads", numThreads))) - .collect(Collectors.toList()); - - if (numThreads > 0) { - threadPool = Executors.newFixedThreadPool(numThreads, - new ThreadFactoryBuilder() - .setDaemon(true) - .setNameFormat(getMethodName() + "-pool-%d") - .build()); - submitter = new PoolSubmitter(); - } else { - submitter = null; - } - - } - - @After - public void teardown() { - if (threadPool != null) { - threadPool.shutdown(); - threadPool = null; - } - } - - private class PoolSubmitter implements Tasks.Submitter { - - @Override - public Future submit(final Runnable task) { - return threadPool.submit(task); - } - - } - - /** - * create the builder. - * @return pre-inited builder - */ - private Tasks.Builder builder() { - return Tasks.foreach(items).executeWith(submitter); - } - - private void assertRun(Tasks.Builder builder, - CounterTask task) throws IOException { - boolean b = builder.run(task); - assertTrue("Run of " + task + " failed", b); - } - - private void assertFailed(Tasks.Builder builder, - CounterTask task) throws IOException { - boolean b = builder.run(task); - assertFalse("Run of " + task + " unexpectedly succeeded", b); - } - - private String itemsToString() { - return "[" + items.stream().map(Item::toString) - .collect(Collectors.joining("\n")) +"]"; - } - - @Test - public void testSimpleInvocation() throws Throwable { - CounterTask t = new CounterTask("simple", 0, Item::commit); - assertRun(builder(), t); - t.assertInvoked("", ITEM_COUNT); - } - - @Test - public void testFailNoStoppingSuppressed() throws Throwable { - assertFailed(builder().suppressExceptions(), failingTask); - failingTask.assertInvoked("Continued through operations", ITEM_COUNT); - items.forEach(Item::assertCommittedOrFailed); - } - - @Test - public void testFailFastSuppressed() throws Throwable { - assertFailed(builder() - .suppressExceptions() - .stopOnFailure(), - failingTask); - if (isParallel()) { - failingTask.assertInvokedAtLeast("stop fast", FAILPOINT); - } else { - failingTask.assertInvoked("stop fast", FAILPOINT); - } - } - - @Test - public void testFailedCallAbortSuppressed() throws Throwable { - assertFailed(builder() - .stopOnFailure() - .suppressExceptions() - .abortWith(aborter), - failingTask); - failingTask.assertInvokedAtLeast("success", FAILPOINT); - if (!isParallel()) { - aborter.assertInvokedAtLeast("abort", 1); - // all uncommitted items were aborted - items.stream().filter(i -> !i.committed) - .map(Item::assertAborted); - items.stream().filter(i -> i.committed) - .forEach(i -> assertFalse(i.toString(), i.aborted)); - } - } - - @Test - public void testFailedCalledWhenNotStoppingSuppressed() throws Throwable { - assertFailed(builder() - .suppressExceptions() - .onFailure(failures), - failingTask); - failingTask.assertInvokedAtLeast("success", FAILPOINT); - // only one failure was triggered - failures.assertInvoked("failure event", 1); - } - - @Test - public void testFailFastCallRevertSuppressed() throws Throwable { - assertFailed(builder() - .stopOnFailure() - .revertWith(reverter) - .abortWith(aborter) - .suppressExceptions() - .onFailure(failures), - failingTask); - failingTask.assertInvokedAtLeast("success", FAILPOINT); - if (!isParallel()) { - aborter.assertInvokedAtLeast("abort", 1); - // all uncommitted items were aborted - items.stream().filter(i -> !i.committed) - .filter(i -> !i.failed) - .forEach(Item::assertAborted); - } - // all committed were reverted - items.stream().filter(i -> i.committed && !i.failed) - .forEach(Item::assertReverted); - // all reverted items are committed - items.stream().filter(i -> i.reverted) - .forEach(Item::assertCommitted); - - // only one failure was triggered - failures.assertInvoked("failure event", 1); - } - - @Test - public void testFailSlowCallRevertSuppressed() throws Throwable { - assertFailed(builder() - .suppressExceptions() - .revertWith(reverter) - .onFailure(failures), - failingTask); - failingTask.assertInvokedAtLeast("success", FAILPOINT); - // all committed were reverted - // identify which task failed from the set - int failing = failures.getItem().id; - items.stream() - .filter(i -> i.id != failing) - .filter(i -> i.committed) - .forEach(Item::assertReverted); - // all reverted items are committed - items.stream().filter(i -> i.reverted) - .forEach(Item::assertCommitted); - - // only one failure was triggered - failures.assertInvoked("failure event", 1); - } - - @Test - public void testFailFastExceptions() throws Throwable { - intercept(IOException.class, - () -> builder() - .stopOnFailure() - .run(failingTask)); - if (isParallel()) { - failingTask.assertInvokedAtLeast("stop fast", FAILPOINT); - } else { - failingTask.assertInvoked("stop fast", FAILPOINT); - } - } - - @Test - public void testFailSlowExceptions() throws Throwable { - intercept(IOException.class, - () -> builder() - .run(failingTask)); - failingTask.assertInvoked("continued through operations", ITEM_COUNT); - items.forEach(Item::assertCommittedOrFailed); - } - - @Test - public void testFailFastExceptionsWithAbortFailure() throws Throwable { - CounterTask failFirst = new CounterTask("task", 1, Item::commit); - CounterTask a = new CounterTask("aborter", 1, Item::abort); - intercept(IOException.class, - () -> builder() - .stopOnFailure() - .abortWith(a) - .run(failFirst)); - if (!isParallel()) { - // expect the other tasks to be aborted - a.assertInvokedAtLeast("abort", ITEM_COUNT - 1); - } - } - - @Test - public void testFailFastExceptionsWithAbortFailureStopped() throws Throwable { - CounterTask failFirst = new CounterTask("task", 1, Item::commit); - CounterTask a = new CounterTask("aborter", 1, Item::abort); - intercept(IOException.class, - () -> builder() - .stopOnFailure() - .stopAbortsOnFailure() - .abortWith(a) - .run(failFirst)); - if (!isParallel()) { - // expect the other tasks to be aborted - a.assertInvoked("abort", 1); - } - } - - /** - * Fail the last one committed, all the rest will be reverted. - * The actual ID of the last task has to be picke dup from the - * failure callback, as in the pool it may be one of any. - */ - @Test - public void testRevertAllSuppressed() throws Throwable { - CounterTask failLast = new CounterTask("task", ITEM_COUNT, Item::commit); - - assertFailed(builder() - .suppressExceptions() - .stopOnFailure() - .revertWith(reverter) - .abortWith(aborter) - .onFailure(failures), - failLast); - failLast.assertInvoked("success", ITEM_COUNT); - int abCount = aborter.getCount(); - int revCount = reverter.getCount(); - assertEquals(ITEM_COUNT, 1 + abCount + revCount); - // identify which task failed from the set - int failing = failures.getItem().id; - // all committed were reverted - items.stream() - .filter(i -> i.id != failing) - .filter(i -> i.committed) - .forEach(Item::assertReverted); - items.stream() - .filter(i -> i.id != failing) - .filter(i -> !i.committed) - .forEach(Item::assertAborted); - // all reverted items are committed - items.stream().filter(i -> i.reverted) - .forEach(Item::assertCommitted); - - // only one failure was triggered - failures.assertInvoked("failure event", 1); - } - - - /** - * The Item which tasks process. - */ - private final class Item { - private final int id; - private final String text; - - private volatile boolean committed, aborted, reverted, failed; - - private Item(int item, String text) { - this.id = item; - this.text = text; - } - - boolean commit() { - committed = true; - return true; - } - - boolean abort() { - aborted = true; - return true; - } - - boolean revert() { - reverted = true; - return true; - } - - boolean fail() { - failed = true; - return true; - } - - public Item assertCommitted() { - assertTrue(toString() + " was not committed in\n" - + itemsToString(), - committed); - return this; - } - - public Item assertCommittedOrFailed() { - assertTrue(toString() + " was not committed nor failed in\n" - + itemsToString(), - committed || failed); - return this; - } - - public Item assertAborted() { - assertTrue(toString() + " was not aborted in\n" - + itemsToString(), - aborted); - return this; - } - - public Item assertReverted() { - assertTrue(toString() + " was not reverted in\n" - + itemsToString(), - reverted); - return this; - } - - @Override - public String toString() { - final StringBuilder sb = new StringBuilder("Item{"); - sb.append(String.format("[%02d]", id)); - sb.append(", committed=").append(committed); - sb.append(", aborted=").append(aborted); - sb.append(", reverted=").append(reverted); - sb.append(", failed=").append(failed); - sb.append(", text=").append(text); - sb.append('}'); - return sb.toString(); - } - } - - /** - * Class which can count invocations and, if limit > 0, will raise - * an exception on the specific invocation of {@link #note(Object)} - * whose count == limit. - */ - private class BaseCounter { - private final AtomicInteger counter = new AtomicInteger(0); - private final int limit; - private final String name; - private Item item; - private final Optional> action; - - /** - * Base counter, tracks items. - * @param name name for string/exception/logs. - * @param limit limit at which an exception is raised, 0 == never - * @param action optional action to invoke after the increment, - * before limit check - */ - BaseCounter(String name, - int limit, - Function action) { - this.name = name; - this.limit = limit; - this.action = Optional.ofNullable(action); - } - - /** - * Apply the action to an item; log at info afterwards with both the - * before and after string values of the item. - * @param i item to process. - * @throws IOException failure in the action - */ - void process(Item i) throws IOException { - this.item = i; - int count = counter.incrementAndGet(); - if (limit == count) { - i.fail(); - LOG.info("{}: Failed {}", this, i); - throw new IOException(String.format("%s: Limit %d reached for %s", - this, limit, i)); - } - String before = i.toString(); - action.map(a -> a.apply(i)); - LOG.info("{}: {} -> {}", this, before, i); - } - - int getCount() { - return counter.get(); - } - - Item getItem() { - return item; - } - - void assertInvoked(String text, int expected) { - assertEquals(toString() + ": " + text, expected, getCount()); - } - - void assertInvokedAtLeast(String text, int expected) { - int actual = getCount(); - assertTrue(toString() + ": " + text - + "-expected " + expected - + " invocations, but got " + actual - + " in " + itemsToString(), - expected <= actual); - } - - @Override - public String toString() { - final StringBuilder sb = new StringBuilder( - "BaseCounter{"); - sb.append("name='").append(name).append('\''); - sb.append(", count=").append(counter.get()); - sb.append(", limit=").append(limit); - sb.append(", item=").append(item); - sb.append('}'); - return sb.toString(); - } - } - - private final class CounterTask - extends BaseCounter implements Tasks.Task { - - private CounterTask(String name, int limit, - Function action) { - super(name, limit, action); - } - - @Override - public void run(Item item) throws IOException { - process(item); - } - - } - - private final class FailureCounter - extends BaseCounter implements Tasks.FailureTask { - private Exception exception; - - private FailureCounter(String name, int limit, - Function action) { - super(name, limit, action); - } - - @Override - public void run(Item item, Exception ex) throws IOException { - process(item); - this.exception = ex; - } - - private Exception getException() { - return exception; - } - - } - -} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocol.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocol.java index 5265163d83..32cab03770 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocol.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocol.java @@ -25,8 +25,6 @@ import org.assertj.core.api.Assertions; import org.junit.Test; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; @@ -34,7 +32,6 @@ import org.apache.hadoop.fs.s3a.commit.AbstractITCommitProtocol; import org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter; import org.apache.hadoop.fs.s3a.commit.CommitConstants; -import org.apache.hadoop.fs.s3a.commit.CommitOperations; import org.apache.hadoop.fs.s3a.commit.CommitUtils; import org.apache.hadoop.fs.s3a.commit.CommitterFaultInjection; import org.apache.hadoop.fs.s3a.commit.CommitterFaultInjectionImpl; @@ -45,7 +42,7 @@ import static org.apache.hadoop.fs.s3a.S3AUtils.listAndFilter; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; -import static org.hamcrest.CoreMatchers.containsString; +import static org.apache.hadoop.util.functional.RemoteIterators.toList; /** * Test the magic committer's commit protocol. @@ -99,8 +96,9 @@ public MagicS3GuardCommitter createFailingCommitter( protected void validateTaskAttemptPathDuringWrite(Path p, final long expectedLength) throws IOException { String pathStr = p.toString(); - assertTrue("not magic " + pathStr, - pathStr.contains(MAGIC)); + Assertions.assertThat(pathStr) + .describedAs("Magic path") + .contains(MAGIC); assertPathDoesNotExist("task attempt visible", p); } @@ -116,9 +114,9 @@ protected void validateTaskAttemptPathAfterWrite(Path marker, // if you list the parent dir and find the marker, it // is really 0 bytes long String name = marker.getName(); - List filtered = listAndFilter(fs, + List filtered = toList(listAndFilter(fs, marker.getParent(), false, - (path) -> path.getName().equals(name)); + (path) -> path.getName().equals(name))); Assertions.assertThat(filtered) .hasSize(1); Assertions.assertThat(filtered.get(0)) @@ -126,14 +124,7 @@ protected void validateTaskAttemptPathAfterWrite(Path marker, "Listing should return 0 byte length"); // marker file is empty - FileStatus st = fs.getFileStatus(marker); - assertEquals("file length in " + st, 0, st.getLen()); - // xattr header - Assertions.assertThat(CommitOperations.extractMagicFileLength(fs, - marker)) - .describedAs("XAttribute " + XA_MAGIC_MARKER) - .isNotEmpty() - .hasValue(expectedLength); + getTestHelper().assertIsMarkerFile(marker, expectedLength); } /** @@ -151,8 +142,8 @@ protected void validateTaskAttemptWorkingDirectory( assertEquals("Wrong schema for working dir " + wd + " with committer " + committer, "s3a", wd.getScheme()); - assertThat(wd.getPath(), - containsString('/' + CommitConstants.MAGIC + '/')); + Assertions.assertThat(wd.getPath()) + .contains('/' + CommitConstants.MAGIC + '/'); } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestS3AHugeMagicCommits.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestS3AHugeMagicCommits.java index 3c15454e7e..aeea195aac 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestS3AHugeMagicCommits.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestS3AHugeMagicCommits.java @@ -20,8 +20,8 @@ import java.io.IOException; import java.util.List; -import java.util.stream.Collectors; +import org.assertj.core.api.Assertions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,10 +33,11 @@ import org.apache.hadoop.fs.s3a.Constants; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.commit.CommitConstants; -import org.apache.hadoop.fs.s3a.commit.CommitOperations; import org.apache.hadoop.fs.s3a.commit.CommitUtils; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; +import org.apache.hadoop.fs.s3a.commit.impl.CommitContext; +import org.apache.hadoop.fs.s3a.commit.impl.CommitOperations; import org.apache.hadoop.fs.s3a.scale.AbstractSTestS3AHugeFiles; import static org.apache.hadoop.fs.s3a.MultipartTestUtils.listMultipartUploads; @@ -54,6 +55,7 @@ public class ITestS3AHugeMagicCommits extends AbstractSTestS3AHugeFiles { private static final Logger LOG = LoggerFactory.getLogger( ITestS3AHugeMagicCommits.class); + private static final int COMMITTER_THREADS = 64; private Path magicDir; private Path jobDir; @@ -123,16 +125,16 @@ public void test_030_postCreationAssertions() throws Throwable { Path destDir = getHugefile().getParent(); assertPathExists("Magic dir", new Path(destDir, CommitConstants.MAGIC)); String destDirKey = fs.pathToKey(destDir); - List uploads = listMultipartUploads(fs, destDirKey); - assertEquals("Pending uploads: " - + uploads.stream() - .collect(Collectors.joining("\n")), 1, uploads.size()); + Assertions.assertThat(listMultipartUploads(fs, destDirKey)) + .describedAs("Pending uploads") + .hasSize(1); assertNotNull("jobDir", jobDir); - Pair>> - results = operations.loadSinglePendingCommits(jobDir, false); - try(CommitOperations.CommitContext commitContext - = operations.initiateCommitOperation(jobDir)) { + try(CommitContext commitContext + = operations.createCommitContextForTesting(jobDir, null, COMMITTER_THREADS)) { + Pair>> + results = operations.loadSinglePendingCommits(jobDir, false, commitContext + ); for (SinglePendingCommit singlePendingCommit : results.getKey().getCommits()) { commitContext.commitOrFail(singlePendingCommit); @@ -140,10 +142,9 @@ public void test_030_postCreationAssertions() throws Throwable { } timer.end("time to commit %s", pendingDataFile); // upload is no longer pending - uploads = listMultipartUploads(fs, destDirKey); - assertEquals("Pending uploads" - + uploads.stream().collect(Collectors.joining("\n")), - 0, operations.listPendingUploadsUnderPath(destDir).size()); + Assertions.assertThat(operations.listPendingUploadsUnderPath(destDir)) + .describedAs("Pending uploads undedr path") + .isEmpty(); // at this point, the huge file exists, so the normal assertions // on that file must be valid. Verify. super.test_030_postCreationAssertions(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/MockedStagingCommitter.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/MockedStagingCommitter.java index d3da8185c8..beccf4f328 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/MockedStagingCommitter.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/MockedStagingCommitter.java @@ -26,6 +26,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.MockS3AFileSystem; +import org.apache.hadoop.fs.s3a.commit.files.SuccessData; import org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.ClientErrors; import org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.ClientResults; import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; @@ -76,11 +77,12 @@ public void commitJob(JobContext context) throws IOException { } @Override - protected void maybeCreateSuccessMarker(JobContext context, + protected SuccessData maybeCreateSuccessMarker(JobContext context, List filenames, final IOStatisticsSnapshot ioStatistics) throws IOException { //skipped + return null; } public ClientResults getResults() throws IOException { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/StagingTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/StagingTestBase.java index 6e13fd0227..6f29537624 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/StagingTestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/StagingTestBase.java @@ -105,10 +105,15 @@ public class StagingTestBase { /** The raw bucket URI Path before any canonicalization. */ public static final URI RAW_BUCKET_URI = RAW_BUCKET_PATH.toUri(); - public static Path outputPath = + + @SuppressWarnings("StaticNonFinalField") + private static Path outputPath = new Path("s3a://" + BUCKET + "/" + OUTPUT_PREFIX); - public static URI outputPathUri = outputPath.toUri(); - public static Path root; + + @SuppressWarnings("StaticNonFinalField") + private static URI outputPathUri = getOutputPath().toUri(); + @SuppressWarnings("StaticNonFinalField") + private static Path root; protected StagingTestBase() { } @@ -131,8 +136,8 @@ protected static S3AFileSystem createAndBindMockFSInstance(Configuration conf, URI uri = RAW_BUCKET_URI; wrapperFS.initialize(uri, conf); root = wrapperFS.makeQualified(new Path("/")); - outputPath = new Path(root, OUTPUT_PREFIX); - outputPathUri = outputPath.toUri(); + outputPath = new Path(getRoot(), OUTPUT_PREFIX); + outputPathUri = getOutputPath().toUri(); FileSystemTestHelper.addFileSystemForTesting(uri, conf, wrapperFS); return mockFs; } @@ -154,7 +159,7 @@ private static S3AFileSystem mockS3AFileSystemRobustly() { */ public static MockS3AFileSystem lookupWrapperFS(Configuration conf) throws IOException { - return (MockS3AFileSystem) FileSystem.get(outputPathUri, conf); + return (MockS3AFileSystem) FileSystem.get(getOutputPathUri(), conf); } public static void verifyCompletion(FileSystem mockS3) throws IOException { @@ -169,13 +174,13 @@ public static void verifyDeleted(FileSystem mockS3, Path path) public static void verifyDeleted(FileSystem mockS3, String child) throws IOException { - verifyDeleted(mockS3, new Path(outputPath, child)); + verifyDeleted(mockS3, new Path(getOutputPath(), child)); } public static void verifyCleanupTempFiles(FileSystem mockS3) throws IOException { verifyDeleted(mockS3, - new Path(outputPath, CommitConstants.TEMPORARY)); + new Path(getOutputPath(), CommitConstants.TEMPORARY)); } protected static void assertConflictResolution( @@ -189,7 +194,7 @@ protected static void assertConflictResolution( public static void pathsExist(FileSystem mockS3, String... children) throws IOException { for (String child : children) { - pathExists(mockS3, new Path(outputPath, child)); + pathExists(mockS3, new Path(getOutputPath(), child)); } } @@ -231,7 +236,7 @@ public static void mkdirsHasOutcome(FileSystem mockS3, public static void canDelete(FileSystem mockS3, String... children) throws IOException { for (String child : children) { - canDelete(mockS3, new Path(outputPath, child)); + canDelete(mockS3, new Path(getOutputPath(), child)); } } @@ -243,7 +248,7 @@ public static void canDelete(FileSystem mockS3, Path f) throws IOException { public static void verifyExistenceChecked(FileSystem mockS3, String child) throws IOException { - verifyExistenceChecked(mockS3, new Path(outputPath, child)); + verifyExistenceChecked(mockS3, new Path(getOutputPath(), child)); } public static void verifyExistenceChecked(FileSystem mockS3, Path path) @@ -262,6 +267,18 @@ public static void verifyMkdirsInvoked(FileSystem mockS3, Path path) verify(mockS3).mkdirs(path); } + protected static URI getOutputPathUri() { + return outputPathUri; + } + + static Path getRoot() { + return root; + } + + static Path getOutputPath() { + return outputPath; + } + /** * Provides setup/teardown of a MiniDFSCluster for tests that need one. */ diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestDirectoryCommitterScale.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestDirectoryCommitterScale.java index cb7202b10d..4d24c07dac 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestDirectoryCommitterScale.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestDirectoryCommitterScale.java @@ -28,6 +28,7 @@ import java.util.stream.IntStream; import com.amazonaws.services.s3.model.PartETag; + import org.apache.hadoop.thirdparty.com.google.common.collect.Maps; import org.assertj.core.api.Assertions; import org.junit.AfterClass; @@ -48,18 +49,21 @@ import org.apache.hadoop.fs.s3a.commit.CommitConstants; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; +import org.apache.hadoop.fs.s3a.commit.impl.CommitContext; +import org.apache.hadoop.fs.s3a.commit.impl.CommitOperations; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.JobStatus; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.util.DurationInfo; +import org.apache.hadoop.util.JsonSerialization; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.CONFLICT_MODE_APPEND; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_STAGING_CONFLICT_MODE; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.PENDINGSET_SUFFIX; import static org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.BUCKET; -import static org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.outputPath; -import static org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.outputPathUri; +import static org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.getOutputPath; +import static org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.getOutputPathUri; import static org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.pathIsDirectory; /** @@ -83,6 +87,7 @@ public class TestDirectoryCommitterScale public static final int TOTAL_COMMIT_COUNT = FILES_PER_TASK * TASKS; public static final int BLOCKS_PER_TASK = 1000; + private static final int COMMITTER_THREAD_COUNT = 100; private static File stagingDir; @@ -95,13 +100,13 @@ public class TestDirectoryCommitterScale @Override DirectoryCommitterForTesting newJobCommitter() throws Exception { - return new DirectoryCommitterForTesting(outputPath, + return new DirectoryCommitterForTesting(getOutputPath(), createTaskAttemptForJob()); } @BeforeClass public static void setupStaging() throws Exception { - stagingDir = File.createTempFile("staging", ""); + stagingDir = File.createTempFile("staging", null); stagingDir.delete(); stagingDir.mkdir(); stagingPath = new Path(stagingDir.toURI()); @@ -125,7 +130,7 @@ protected JobConf createJobConf() { JobConf conf = super.createJobConf(); conf.setInt( CommitConstants.FS_S3A_COMMITTER_THREADS, - 100); + COMMITTER_THREAD_COUNT); return conf; } @@ -149,7 +154,8 @@ public void test_010_createTaskFiles() throws Exception { */ private void createTasks() throws IOException { // create a stub multipart commit containing multiple files. - + JsonSerialization serializer = + SinglePendingCommit.serializer(); // step1: a list of tags. // this is the md5sum of hadoop 3.2.1.tar String tag = "9062dcf18ffaee254821303bbd11c72b"; @@ -164,12 +170,14 @@ private void createTasks() throws IOException { // these get overwritten base.setDestinationKey("/base"); base.setUploadId("uploadId"); - base.setUri(outputPathUri.toString()); + base.setUri(getOutputPathUri().toString()); + byte[] bytes = base.toBytes(serializer); SinglePendingCommit[] singles = new SinglePendingCommit[FILES_PER_TASK]; - byte[] bytes = base.toBytes(); + for (int i = 0; i < FILES_PER_TASK; i++) { - singles[i] = SinglePendingCommit.serializer().fromBytes(bytes); + + singles[i] = serializer.fromBytes(bytes); } // now create the files, using this as the template @@ -182,7 +190,7 @@ private void createTasks() throws IOException { String uploadId = String.format("%05d-task-%04d-file-%02d", uploadCount, task, i); // longer paths to take up more space. - Path p = new Path(outputPath, + Path p = new Path(getOutputPath(), "datasets/examples/testdirectoryscale/" + "year=2019/month=09/day=26/hour=20/second=53" + uploadId); @@ -199,7 +207,7 @@ private void createTasks() throws IOException { } Path path = new Path(stagingPath, String.format("task-%04d." + PENDINGSET_SUFFIX, task)); - pending.save(localFS, path, true); + pending.save(localFS, path, PendingSet.serializer()); } } @@ -211,12 +219,14 @@ public void test_020_loadFilesToAttempt() throws Exception { Configuration jobConf = getJobConf(); jobConf.set( FS_S3A_COMMITTER_STAGING_CONFLICT_MODE, CONFLICT_MODE_APPEND); - FileSystem mockS3 = getMockS3A(); - pathIsDirectory(mockS3, outputPath); - try (DurationInfo ignored = - new DurationInfo(LOG, "listing pending uploads")) { + S3AFileSystem mockS3 = getMockS3A(); + pathIsDirectory(mockS3, getOutputPath()); + final CommitOperations operations = new CommitOperations(getWrapperFS()); + try (CommitContext commitContext + = operations.createCommitContextForTesting(getOutputPath(), + null, COMMITTER_THREAD_COUNT)) { AbstractS3ACommitter.ActiveCommit activeCommit - = committer.listPendingUploadsToCommit(getJob()); + = committer.listPendingUploadsToCommit(commitContext); Assertions.assertThat(activeCommit.getSourceFiles()) .describedAs("Source files of %s", activeCommit) .hasSize(TASKS); @@ -232,7 +242,7 @@ public void test_030_commitFiles() throws Exception { jobConf.set( FS_S3A_COMMITTER_STAGING_CONFLICT_MODE, CONFLICT_MODE_APPEND); S3AFileSystem mockS3 = getMockS3A(); - pathIsDirectory(mockS3, outputPath); + pathIsDirectory(mockS3, getOutputPath()); try (DurationInfo ignored = new DurationInfo(LOG, "Committing Job")) { @@ -261,7 +271,7 @@ public void test_040_abortFiles() throws Exception { jobConf.set( FS_S3A_COMMITTER_STAGING_CONFLICT_MODE, CONFLICT_MODE_APPEND); FileSystem mockS3 = getMockS3A(); - pathIsDirectory(mockS3, outputPath); + pathIsDirectory(mockS3, getOutputPath()); committer.abortJob(getJob(), JobStatus.State.FAILED); } @@ -304,11 +314,11 @@ public Path getJobAttemptPath(JobContext context) { } @Override - protected void commitJobInternal(final JobContext context, + protected void commitJobInternal(final CommitContext commitContext, final ActiveCommit pending) throws IOException { activeCommit = pending; - super.commitJobInternal(context, pending); + super.commitJobInternal(commitContext, pending); } } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingCommitter.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingCommitter.java index 94c0b29a3c..11edf0d216 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingCommitter.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingCommitter.java @@ -37,7 +37,6 @@ import org.apache.hadoop.util.Sets; import org.assertj.core.api.Assertions; -import org.hamcrest.core.StringStartsWith; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -58,6 +57,7 @@ import org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter; import org.apache.hadoop.fs.s3a.commit.PathCommitException; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; +import org.apache.hadoop.fs.s3a.commit.files.PersistentCommitData; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapreduce.JobContext; @@ -168,7 +168,7 @@ public void setupCommitter() throws Exception { this.tac = new TaskAttemptContextImpl( new Configuration(job.getConfiguration()), AID); - this.jobCommitter = new MockedStagingCommitter(outputPath, tac); + this.jobCommitter = new MockedStagingCommitter(getOutputPath(), tac); jobCommitter.setupJob(job); // get the task's configuration copy so modifications take effect @@ -183,7 +183,7 @@ public void setupCommitter() throws Exception { this.conf.set(BUFFER_DIR, String.format("%s/local-0/, %s/local-1 ", tmp, tmp)); - this.committer = new MockedStagingCommitter(outputPath, tac); + this.committer = new MockedStagingCommitter(getOutputPath(), tac); Paths.resetTempFolderCache(); } @@ -335,10 +335,11 @@ public void testAttemptPathConstructionWithSchema() throws Exception { config.set(BUFFER_DIR, "file:/tmp/mr-local-0,file:/tmp/mr-local-1"); - assertThat("Path should be the same with file scheme", + Assertions.assertThat( getLocalTaskAttemptTempDir(config, - jobUUID, tac.getTaskAttemptID()).toString(), - StringStartsWith.startsWith(commonPath)); + jobUUID, tac.getTaskAttemptID()).toString()) + .describedAs("Path should be the same with file scheme") + .startsWith(commonPath); } @Test @@ -379,7 +380,7 @@ public void testSingleTaskCommit() throws Exception { assertEquals("Should name the commits file with the task ID: " + results, "task_job_0001_r_000002", stats[0].getPath().getName()); - PendingSet pending = PendingSet.load(dfs, stats[0]); + PendingSet pending = PersistentCommitData.load(dfs, stats[0], PendingSet.serializer()); assertEquals("Should have one pending commit", 1, pending.size()); SinglePendingCommit commit = pending.getCommits().get(0); assertEquals("Should write to the correct bucket:" + results, @@ -419,7 +420,7 @@ public void testSingleTaskEmptyFileCommit() throws Exception { assertEquals("Should name the commits file with the task ID", "task_job_0001_r_000002", stats[0].getPath().getName()); - PendingSet pending = PendingSet.load(dfs, stats[0]); + PendingSet pending = PersistentCommitData.load(dfs, stats[0], PendingSet.serializer()); assertEquals("Should have one pending commit", 1, pending.size()); } @@ -442,7 +443,7 @@ public void testSingleTaskMultiFileCommit() throws Exception { "task_job_0001_r_000002", stats[0].getPath().getName()); List pending = - PendingSet.load(dfs, stats[0]).getCommits(); + PersistentCommitData.load(dfs, stats[0], PendingSet.serializer()).getCommits(); assertEquals("Should have correct number of pending commits", files.size(), pending.size()); @@ -717,7 +718,7 @@ private Set runTasks(JobContext jobContext, TaskAttemptContext attempt = new TaskAttemptContextImpl( new Configuration(jobContext.getConfiguration()), attemptID); MockedStagingCommitter taskCommitter = new MockedStagingCommitter( - outputPath, attempt); + getOutputPath(), attempt); commitTask(taskCommitter, attempt, numFiles); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingDirectoryOutputCommitter.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingDirectoryOutputCommitter.java index 98075b827a..b92605ca25 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingDirectoryOutputCommitter.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingDirectoryOutputCommitter.java @@ -29,8 +29,11 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.PathExistsException; +import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter; import org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants; +import org.apache.hadoop.fs.s3a.commit.impl.CommitContext; +import org.apache.hadoop.fs.s3a.commit.impl.CommitOperations; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; import static org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.*; @@ -46,7 +49,7 @@ public class TestStagingDirectoryOutputCommitter @Override DirectoryStagingCommitter newJobCommitter() throws Exception { - return new DirectoryStagingCommitter(outputPath, + return new DirectoryStagingCommitter(getOutputPath(), createTaskAttemptForJob()); } @@ -63,7 +66,7 @@ public void testBadConflictMode() throws Throwable { public void testDefaultConflictResolution() throws Exception { getJob().getConfiguration().unset( FS_S3A_COMMITTER_STAGING_CONFLICT_MODE); - pathIsDirectory(getMockS3A(), outputPath); + pathIsDirectory(getMockS3A(), getOutputPath()); verifyJobSetupAndCommit(); } @@ -75,7 +78,8 @@ public void testFailConflictResolution() throws Exception { } protected void verifyFailureConflictOutcome() throws Exception { - pathIsDirectory(getMockS3A(), outputPath); + final S3AFileSystem mockFS = getMockS3A(); + pathIsDirectory(mockFS, getOutputPath()); final DirectoryStagingCommitter committer = newJobCommitter(); // this should fail @@ -86,20 +90,23 @@ protected void verifyFailureConflictOutcome() throws Exception { // but there are no checks in job commit (HADOOP-15469) // this is done by calling the preCommit method directly, - committer.preCommitJob(getJob(), AbstractS3ACommitter.ActiveCommit.empty()); - reset(getMockS3A()); - pathDoesNotExist(getMockS3A(), outputPath); + final CommitContext commitContext = new CommitOperations(getWrapperFS()). + createCommitContext(getJob(), getOutputPath(), 0); + committer.preCommitJob(commitContext, AbstractS3ACommitter.ActiveCommit.empty()); + + reset(mockFS); + pathDoesNotExist(mockFS, getOutputPath()); committer.setupJob(getJob()); - verifyExistenceChecked(getMockS3A(), outputPath); - verifyMkdirsInvoked(getMockS3A(), outputPath); - verifyNoMoreInteractions(getMockS3A()); + verifyExistenceChecked(mockFS, getOutputPath()); + verifyMkdirsInvoked(mockFS, getOutputPath()); + verifyNoMoreInteractions(mockFS); - reset(getMockS3A()); - pathDoesNotExist(getMockS3A(), outputPath); + reset(mockFS); + pathDoesNotExist(mockFS, getOutputPath()); committer.commitJob(getJob()); - verifyCompletion(getMockS3A()); + verifyCompletion(mockFS); } @Test @@ -108,7 +115,7 @@ public void testAppendConflictResolution() throws Exception { getJob().getConfiguration().set( FS_S3A_COMMITTER_STAGING_CONFLICT_MODE, CONFLICT_MODE_APPEND); FileSystem mockS3 = getMockS3A(); - pathIsDirectory(mockS3, outputPath); + pathIsDirectory(mockS3, getOutputPath()); verifyJobSetupAndCommit(); } @@ -120,7 +127,7 @@ protected void verifyJobSetupAndCommit() FileSystem mockS3 = getMockS3A(); Mockito.reset(mockS3); - pathExists(mockS3, outputPath); + pathExists(mockS3, getOutputPath()); committer.commitJob(getJob()); verifyCompletion(mockS3); @@ -130,7 +137,7 @@ protected void verifyJobSetupAndCommit() public void testReplaceConflictResolution() throws Exception { FileSystem mockS3 = getMockS3A(); - pathIsDirectory(mockS3, outputPath); + pathIsDirectory(mockS3, getOutputPath()); getJob().getConfiguration().set( FS_S3A_COMMITTER_STAGING_CONFLICT_MODE, CONFLICT_MODE_REPLACE); @@ -140,17 +147,17 @@ public void testReplaceConflictResolution() throws Exception { committer.setupJob(getJob()); Mockito.reset(mockS3); - pathExists(mockS3, outputPath); - canDelete(mockS3, outputPath); + pathExists(mockS3, getOutputPath()); + canDelete(mockS3, getOutputPath()); committer.commitJob(getJob()); - verifyDeleted(mockS3, outputPath); + verifyDeleted(mockS3, getOutputPath()); verifyCompletion(mockS3); } @Test public void testReplaceConflictFailsIfDestIsFile() throws Exception { - pathIsFile(getMockS3A(), outputPath); + pathIsFile(getMockS3A(), getOutputPath()); getJob().getConfiguration().set( FS_S3A_COMMITTER_STAGING_CONFLICT_MODE, CONFLICT_MODE_REPLACE); @@ -166,7 +173,7 @@ public void testReplaceConflictFailsIfDestIsFile() throws Exception { @Test public void testAppendConflictFailsIfDestIsFile() throws Exception { - pathIsFile(getMockS3A(), outputPath); + pathIsFile(getMockS3A(), getOutputPath()); getJob().getConfiguration().set( FS_S3A_COMMITTER_STAGING_CONFLICT_MODE, CONFLICT_MODE_APPEND); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedFileListing.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedFileListing.java index 76a0de2253..64a9be0888 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedFileListing.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedFileListing.java @@ -51,13 +51,13 @@ public class TestStagingPartitionedFileListing @Override PartitionedStagingCommitter newJobCommitter() throws IOException { - return new PartitionedStagingCommitter(outputPath, + return new PartitionedStagingCommitter(getOutputPath(), createTaskAttemptForJob()); } @Override PartitionedStagingCommitter newTaskCommitter() throws IOException { - return new PartitionedStagingCommitter(outputPath, getTAC()); + return new PartitionedStagingCommitter(getOutputPath(), getTAC()); } private FileSystem attemptFS; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedJobCommit.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedJobCommit.java index 86b677c70a..28161979f0 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedJobCommit.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedJobCommit.java @@ -34,7 +34,7 @@ import org.apache.hadoop.fs.s3a.commit.PathCommitException; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; -import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.fs.s3a.commit.impl.CommitContext; import org.apache.hadoop.mapreduce.TaskAttemptContext; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -67,7 +67,7 @@ private final class PartitionedStagingCommitterForTesting private PartitionedStagingCommitterForTesting(TaskAttemptContext context) throws IOException { - super(StagingTestBase.outputPath, context); + super(StagingTestBase.getOutputPath(), context); } /** @@ -75,14 +75,15 @@ private PartitionedStagingCommitterForTesting(TaskAttemptContext context) * This is quite complex as the mock pending uploads need to be saved * to a filesystem for the next stage of the commit process. * To simulate multiple commit, more than one .pendingset file is created, - * @param context job context + * @param commitContext job context * @return an active commit containing a list of paths to valid pending set * file. * @throws IOException IO failure */ + @SuppressWarnings("CollectionDeclaredAsConcreteClass") @Override protected ActiveCommit listPendingUploadsToCommit( - JobContext context) throws IOException { + CommitContext commitContext) throws IOException { LocalFileSystem localFS = FileSystem.getLocal(getConf()); ActiveCommit activeCommit = new ActiveCommit(localFS, @@ -109,17 +110,17 @@ protected ActiveCommit listPendingUploadsToCommit( File file = File.createTempFile("staging", ".pendingset"); file.deleteOnExit(); Path path = new Path(file.toURI()); - pendingSet.save(localFS, path, true); + pendingSet.save(localFS, path, PendingSet.serializer()); activeCommit.add(localFS.getFileStatus(path)); } return activeCommit; } @Override - protected void abortJobInternal(JobContext context, + protected void abortJobInternal(CommitContext commitContext, boolean suppressExceptions) throws IOException { this.aborted = true; - super.abortJobInternal(context, suppressExceptions); + super.abortJobInternal(commitContext, suppressExceptions); } } @@ -242,7 +243,7 @@ public void testReplaceWithDeleteFailure() throws Exception { pathsExist(mockS3, "dateint=20161116/hour=14"); when(mockS3 .delete( - new Path(outputPath, "dateint=20161116/hour=14"), + new Path(getOutputPath(), "dateint=20161116/hour=14"), true)) .thenThrow(new PathCommitException("fake", "Fake IOException for delete")); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedTaskCommit.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedTaskCommit.java index fb25210249..4e82b94314 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedTaskCommit.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedTaskCommit.java @@ -47,13 +47,13 @@ public class TestStagingPartitionedTaskCommit @Override PartitionedStagingCommitter newJobCommitter() throws IOException { - return new PartitionedStagingCommitter(outputPath, + return new PartitionedStagingCommitter(getOutputPath(), createTaskAttemptForJob()); } @Override PartitionedStagingCommitter newTaskCommitter() throws Exception { - return new PartitionedStagingCommitter(outputPath, getTAC()); + return new PartitionedStagingCommitter(getOutputPath(), getTAC()); } // The set of files used by this test @@ -104,7 +104,7 @@ public void testFail() throws Exception { // test failure when one partition already exists reset(mockS3); - Path existsPath = new Path(outputPath, relativeFiles.get(1)).getParent(); + Path existsPath = new Path(getOutputPath(), relativeFiles.get(1)).getParent(); pathExists(mockS3, existsPath); intercept(PathExistsException.class, "", @@ -133,7 +133,7 @@ public void testAppend() throws Exception { // test success when one partition already exists reset(mockS3); - pathExists(mockS3, new Path(outputPath, relativeFiles.get(2)).getParent()); + pathExists(mockS3, new Path(getOutputPath(), relativeFiles.get(2)).getParent()); committer.commitTask(getTAC()); verifyFilesCreated(committer); @@ -178,7 +178,7 @@ public void testReplace() throws Exception { // test success when one partition already exists reset(mockS3); - pathExists(mockS3, new Path(outputPath, relativeFiles.get(3)).getParent()); + pathExists(mockS3, new Path(getOutputPath(), relativeFiles.get(3)).getParent()); committer.commitTask(getTAC()); verifyFilesCreated(committer); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocol.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocol.java index 3a820bcc11..bb3031b32c 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocol.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocol.java @@ -52,7 +52,7 @@ protected String suitename() { @Override protected Configuration createConfiguration() { Configuration conf = super.createConfiguration(); - conf.setInt(FS_S3A_COMMITTER_THREADS, 1); + conf.setInt(FS_S3A_COMMITTER_THREADS, 4); // disable unique filenames so that the protocol tests of FileOutputFormat // and this test generate consistent names. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/ITestTerasortOnS3A.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/ITestTerasortOnS3A.java index 991969b0f0..d28ee5172b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/ITestTerasortOnS3A.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/ITestTerasortOnS3A.java @@ -330,7 +330,7 @@ public void test_140_teracomplete() throws Throwable { stage.accept("teravalidate"); stage.accept("overall"); String text = results.toString(); - File resultsFile = File.createTempFile("results", ".csv"); + File resultsFile = new File(getReportDir(), committerName + ".csv"); FileUtils.write(resultsFile, text, StandardCharsets.UTF_8); LOG.info("Results are in {}\n{}", resultsFile, text); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/StubContextAccessor.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/StubContextAccessor.java new file mode 100644 index 0000000000..41180667e1 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/StubContextAccessor.java @@ -0,0 +1,82 @@ +/* + * 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.s3a.impl; + +import java.io.File; +import java.io.IOException; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.MockS3AFileSystem; +import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.audit.AuditTestSupport; +import org.apache.hadoop.fs.store.audit.AuditSpan; + +/** + * A Stub context acccessor for test. + */ +public final class StubContextAccessor + implements ContextAccessors { + + private final String bucket; + + /** + * Construct. + * @param bucket bucket to use when qualifying keys.]= + */ + public StubContextAccessor(String bucket) { + this.bucket = bucket; + } + + @Override + public Path keyToPath(final String key) { + return new Path("s3a://" + bucket + "/" + key); + } + + @Override + public String pathToKey(final Path path) { + return null; + } + + @Override + public File createTempFile(final String prefix, final long size) + throws IOException { + throw new UnsupportedOperationException("unsppported"); + } + + @Override + public String getBucketLocation() throws IOException { + return null; + } + + @Override + public Path makeQualified(final Path path) { + return path; + } + + @Override + public AuditSpan getActiveAuditSpan() { + return AuditTestSupport.NOOP_SPAN; + } + + @Override + public RequestFactory getRequestFactory() { + return MockS3AFileSystem.REQUEST_FACTORY; + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestCreateFileBuilder.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestCreateFileBuilder.java new file mode 100644 index 0000000000..65d7aa6192 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestCreateFileBuilder.java @@ -0,0 +1,173 @@ +/* + * 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.s3a.impl; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.Map; + +import org.assertj.core.api.Assertions; +import org.junit.Test; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CreateFlag; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FSDataOutputStreamBuilder; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.test.HadoopTestBase; +import org.apache.hadoop.util.Progressable; + +import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_CREATE_HEADER; +import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_CREATE_PERFORMANCE; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +/** + * Unit test of {@link CreateFileBuilder}. + */ +public class TestCreateFileBuilder extends HadoopTestBase { + + private static final BuilderCallbacks CALLBACKS = new BuilderCallbacks(); + + private CreateFileBuilder mkBuilder() throws IOException { + return new CreateFileBuilder( + FileSystem.getLocal(new Configuration()), + new Path("/"), + CALLBACKS); + } + + private BuilderOutputStream unwrap(FSDataOutputStream out) { + OutputStream s = out.getWrappedStream(); + Assertions.assertThat(s) + .isInstanceOf(BuilderOutputStream.class); + return (BuilderOutputStream) s; + } + + private BuilderOutputStream build(FSDataOutputStreamBuilder builder) + throws IOException { + return unwrap(builder.build()); + } + + @Test + public void testSimpleBuild() throws Throwable { + Assertions.assertThat(build(mkBuilder().create())) + .matches(p -> !p.isOverwrite()) + .matches(p -> !p.isPerformance()); + } + + @Test + public void testAppendForbidden() throws Throwable { + intercept(UnsupportedOperationException.class, () -> + build(mkBuilder().append())); + } + + @Test + public void testPerformanceSupport() throws Throwable { + CreateFileBuilder builder = mkBuilder().create(); + builder.must(FS_S3A_CREATE_PERFORMANCE, true); + Assertions.assertThat(build(builder)) + .matches(p -> p.isPerformance()); + } + + @Test + public void testHeaderOptions() throws Throwable { + final CreateFileBuilder builder = mkBuilder().create() + .must(FS_S3A_CREATE_HEADER + ".retention", "permanent") + .opt(FS_S3A_CREATE_HEADER + ".owner", "engineering"); + final Map headers = build(builder).getHeaders(); + Assertions.assertThat(headers) + .containsEntry("retention", "permanent") + .containsEntry("owner", "engineering"); + } + + @Test + public void testIncompleteHeader() throws Throwable { + final CreateFileBuilder builder = mkBuilder().create() + .must(FS_S3A_CREATE_HEADER, "permanent"); + intercept(IllegalArgumentException.class, () -> + build(builder)); + } + + private static final class BuilderCallbacks implements + CreateFileBuilder.CreateFileBuilderCallbacks { + + @Override + public FSDataOutputStream createFileFromBuilder(final Path path, + final Progressable progress, + final CreateFileBuilder.CreateFileOptions options) throws IOException { + return new FSDataOutputStream( + new BuilderOutputStream( + progress, + options), + null); + } + } + + /** + * Stream which will be wrapped and which returns the flags used + * creating the object. + */ + private static final class BuilderOutputStream extends OutputStream { + + private final Progressable progress; + + + private final CreateFileBuilder.CreateFileOptions options; + + private BuilderOutputStream(final Progressable progress, + final CreateFileBuilder.CreateFileOptions options) { + this.progress = progress; + this.options = options; + } + + private boolean isOverwrite() { + return options.getFlags().contains(CreateFlag.OVERWRITE); + } + + private Progressable getProgress() { + return progress; + } + + private boolean isPerformance() { + return options.isPerformance(); + } + + private CreateFileBuilder.CreateFileOptions getOptions() { + return options; + } + + private Map getHeaders() { + return options.getHeaders(); + } + + @Override + public void write(final int b) throws IOException { + + } + + @Override + public String toString() { + return "BuilderOutputStream{" + + "progress=" + progress + + ", options=" + options + + "} " + super.toString(); + } + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java index 9bc3aef83a..5c243bb820 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java @@ -90,7 +90,7 @@ public void testRequestFactoryWithCannedACL() throws Throwable { ObjectMetadata md = factory.newObjectMetadata(128); Assertions.assertThat( factory.newPutObjectRequest(path, md, - new ByteArrayInputStream(new byte[0])) + null, new ByteArrayInputStream(new byte[0])) .getCannedAcl()) .describedAs("ACL of PUT") .isEqualTo(acl); @@ -98,7 +98,8 @@ public void testRequestFactoryWithCannedACL() throws Throwable { .getCannedAccessControlList()) .describedAs("ACL of COPY") .isEqualTo(acl); - Assertions.assertThat(factory.newMultipartUploadRequest(path) + Assertions.assertThat(factory.newMultipartUploadRequest(path, + null) .getCannedACL()) .describedAs("ACL of MPU") .isEqualTo(acl); @@ -172,12 +173,12 @@ private void createFactoryObjects(RequestFactory factory) { a(factory.newListObjectsV1Request(path, "/", 1)); a(factory.newListNextBatchOfObjectsRequest(new ObjectListing())); a(factory.newListObjectsV2Request(path, "/", 1)); - a(factory.newMultipartUploadRequest(path)); + a(factory.newMultipartUploadRequest(path, null)); File srcfile = new File("/tmp/a"); a(factory.newPutObjectRequest(path, - factory.newObjectMetadata(-1), srcfile)); + factory.newObjectMetadata(-1), null, srcfile)); ByteArrayInputStream stream = new ByteArrayInputStream(new byte[0]); - a(factory.newPutObjectRequest(path, md, stream)); + a(factory.newPutObjectRequest(path, md, null, stream)); a(factory.newSelectRequest(path)); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java index 3511020aa6..8bbf52b578 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java @@ -132,16 +132,7 @@ public void setup() throws Exception { .isEqualTo(isKeepingMarkers() ? DirectoryPolicy.MarkerPolicy.Keep : DirectoryPolicy.MarkerPolicy.Delete); - // All counter statistics of the filesystem are added as metrics. - // Durations too, as they have counters of success and failure. - OperationCostValidator.Builder builder = OperationCostValidator.builder( - getFileSystem()); - EnumSet.allOf(Statistic.class).stream() - .filter(s -> - s.getType() == StatisticTypeEnum.TYPE_COUNTER - || s.getType() == StatisticTypeEnum.TYPE_DURATION) - .forEach(s -> builder.withMetric(s)); - costValidator = builder.build(); + setupCostValidator(); // determine bulk delete settings final Configuration fsConf = getFileSystem().getConf(); @@ -154,6 +145,19 @@ public void setup() throws Exception { setSpanSource(fs); } + protected void setupCostValidator() { + // All counter statistics of the filesystem are added as metrics. + // Durations too, as they have counters of success and failure. + OperationCostValidator.Builder builder = OperationCostValidator.builder( + getFileSystem()); + EnumSet.allOf(Statistic.class).stream() + .filter(s -> + s.getType() == StatisticTypeEnum.TYPE_COUNTER + || s.getType() == StatisticTypeEnum.TYPE_DURATION) + .forEach(s -> builder.withMetric(s)); + costValidator = builder.build(); + } + public boolean isDeleting() { return isDeleting; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestCreateFileCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestCreateFileCost.java new file mode 100644 index 0000000000..39530d97bf --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestCreateFileCost.java @@ -0,0 +1,248 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.performance; + +import java.io.IOException; + +import org.assertj.core.api.Assertions; +import org.junit.Test; + +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FSDataOutputStreamBuilder; +import org.apache.hadoop.fs.FileAlreadyExistsException; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.s3a.S3AFileSystem; + + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.fs.contract.ContractTestUtils.toChar; +import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_CREATE_HEADER; +import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_CREATE_PERFORMANCE; +import static org.apache.hadoop.fs.s3a.Constants.XA_HEADER_PREFIX; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_BULK_DELETE_REQUEST; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_DELETE_REQUEST; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.CREATE_FILE_NO_OVERWRITE; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.CREATE_FILE_OVERWRITE; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.FILE_STATUS_DIR_PROBE; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.FILE_STATUS_FILE_PROBE; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.GET_FILE_STATUS_ON_DIR_MARKER; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.GET_FILE_STATUS_ON_FILE; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.HEAD_OPERATION; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.NO_HEAD_OR_LIST; + +/** + * Assert cost of createFile operations, especially + * with the FS_S3A_CREATE_PERFORMANCE option. + */ +@SuppressWarnings("resource") +public class ITestCreateFileCost extends AbstractS3ACostTest { + + /** + * Create with markers kept, always. + */ + public ITestCreateFileCost() { + super(false); + } + + @Test + public void testCreateNoOverwrite() throws Throwable { + describe("Test file creation without overwrite"); + Path testFile = methodPath(); + // when overwrite is false, the path is checked for existence. + create(testFile, false, + CREATE_FILE_NO_OVERWRITE); + } + + @Test + public void testCreateOverwrite() throws Throwable { + describe("Test file creation with overwrite"); + Path testFile = methodPath(); + // when overwrite is true: only the directory checks take place. + create(testFile, true, CREATE_FILE_OVERWRITE); + } + + @Test + public void testCreateNoOverwriteFileExists() throws Throwable { + describe("Test cost of create file failing with existing file"); + Path testFile = file(methodPath()); + + // now there is a file there, an attempt with overwrite == false will + // fail on the first HEAD. + interceptOperation(FileAlreadyExistsException.class, "", + FILE_STATUS_FILE_PROBE, + () -> file(testFile, false)); + } + + @Test + public void testCreateFileOverDir() throws Throwable { + describe("Test cost of create file failing with existing dir"); + Path testFile = dir(methodPath()); + + // now there is a file there, an attempt with overwrite == false will + // fail on the first HEAD. + interceptOperation(FileAlreadyExistsException.class, "", + GET_FILE_STATUS_ON_DIR_MARKER, + () -> file(testFile, false)); + } + + /** + * Use the builder API. + * on s3a this skips parent checks, always. + */ + @Test + public void testCreateBuilderSequence() throws Throwable { + describe("Test builder file creation cost"); + Path testFile = methodPath(); + dir(testFile.getParent()); + + // s3a fs skips the recursive checks to avoid race + // conditions with other processes/threads deleting + // files and so briefly the path not being present + // only make sure the dest path isn't a directory. + buildFile(testFile, true, false, + FILE_STATUS_DIR_PROBE); + + // now there is a file there, an attempt with overwrite == false will + // fail on the first HEAD. + interceptOperation(FileAlreadyExistsException.class, "", + GET_FILE_STATUS_ON_FILE, + () -> buildFile(testFile, false, true, + GET_FILE_STATUS_ON_FILE)); + } + + @Test + public void testCreateFilePerformanceFlag() throws Throwable { + describe("createFile with performance flag skips safety checks"); + S3AFileSystem fs = getFileSystem(); + + Path path = methodPath(); + FSDataOutputStreamBuilder builder = fs.createFile(path) + .overwrite(false) + .recursive(); + + // this has a broken return type; something to do with the return value of + // the createFile() call. only fixable via risky changes to the FileSystem class + builder.must(FS_S3A_CREATE_PERFORMANCE, true); + + verifyMetrics(() -> build(builder), + always(NO_HEAD_OR_LIST), + with(OBJECT_BULK_DELETE_REQUEST, 0), + with(OBJECT_DELETE_REQUEST, 0)); + } + + @Test + public void testCreateFileRecursive() throws Throwable { + describe("createFile without performance flag performs overwrite safety checks"); + S3AFileSystem fs = getFileSystem(); + + final Path path = methodPath(); + FSDataOutputStreamBuilder builder = fs.createFile(path) + .recursive() + .overwrite(false); + + // include a custom header to probe for after + final String custom = "custom"; + builder.must(FS_S3A_CREATE_HEADER + ".h1", custom); + + verifyMetrics(() -> build(builder), + always(CREATE_FILE_NO_OVERWRITE)); + + // the header is there and the probe should be a single HEAD call. + String header = verifyMetrics(() -> + toChar(requireNonNull( + fs.getXAttr(path, XA_HEADER_PREFIX + "h1"), + "no header")), + always(HEAD_OPERATION)); + Assertions.assertThat(header) + .isEqualTo(custom); + } + + @Test + public void testCreateFileNonRecursive() throws Throwable { + describe("nonrecursive createFile does not check parents"); + S3AFileSystem fs = getFileSystem(); + + verifyMetrics(() -> + build(fs.createFile(methodPath()).overwrite(true)), + always(CREATE_FILE_OVERWRITE)); + } + + + @Test + public void testCreateNonRecursive() throws Throwable { + describe("nonrecursive createFile does not check parents"); + S3AFileSystem fs = getFileSystem(); + + verifyMetrics(() -> { + fs.createNonRecursive(methodPath(), + true, 1000, (short)1, 0L, null) + .close(); + return ""; + }, + always(CREATE_FILE_OVERWRITE)); + } + + private FSDataOutputStream build(final FSDataOutputStreamBuilder builder) + throws IOException { + FSDataOutputStream out = builder.build(); + out.close(); + return out; + } + + /** + * Shows how the performance option allows the FS to become ill-formed. + */ + @Test + public void testPerformanceFlagPermitsInvalidStores() throws Throwable { + describe("createFile with performance flag over a directory"); + S3AFileSystem fs = getFileSystem(); + + Path path = methodPath(); + Path child = new Path(path, "child"); + ContractTestUtils.touch(fs, child); + try { + FSDataOutputStreamBuilder builder = fs.createFile(path) + .overwrite(false); + // this has a broken return type; a java typesystem quirk. + builder.must(FS_S3A_CREATE_PERFORMANCE, true); + + verifyMetrics(() -> build(builder), + always(NO_HEAD_OR_LIST), + with(OBJECT_BULK_DELETE_REQUEST, 0), + with(OBJECT_DELETE_REQUEST, 0)); + // the file is there + assertIsFile(path); + // the child is there + assertIsFile(child); + + // delete the path + fs.delete(path, true); + // the child is still there + assertIsFile(child); + // and the directory exists again + assertIsDirectory(path); + } finally { + // always delete the child, so if the test suite fails, the + // store is at least well-formed. + fs.delete(child, true); + } + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java index 01cadc7c86..be4de7942f 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java @@ -266,9 +266,10 @@ public void testDirMarkersFileCreation() throws Throwable { final int directories = directoriesInPath(srcDir); verifyMetrics(() -> { - file(new Path(srcDir, "source.txt")); + final Path srcPath = new Path(srcDir, "source.txt"); + file(srcPath); LOG.info("Metrics: {}\n{}", getMetricSummary(), getFileSystem()); - return "after touch(fs, srcFilePath) " + getMetricSummary(); + return "after touch(fs, " + srcPath + ")" + getMetricSummary(); }, with(DIRECTORIES_CREATED, 0), with(DIRECTORIES_DELETED, 0), @@ -276,10 +277,10 @@ public void testDirMarkersFileCreation() throws Throwable { withWhenKeeping(getDeleteMarkerStatistic(), 0), withWhenKeeping(FAKE_DIRECTORIES_DELETED, 0), // delete all possible fake dirs above the file - withWhenDeleting(getDeleteMarkerStatistic(), - isBulkDelete() ? 1: directories), withWhenDeleting(FAKE_DIRECTORIES_DELETED, - directories)); + directories), + withWhenDeleting(getDeleteMarkerStatistic(), + isBulkDelete() ? 1: directories)); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCost.java index 03bc10f86c..08a19600d5 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCost.java @@ -57,7 +57,8 @@ public final class OperationCost { public static final int DELETE_MARKER_REQUEST = DELETE_OBJECT_REQUEST; /** - * No IO takes place. + * No Head or List IO takes place; other operations + * may still take place. */ public static final OperationCost NO_IO = new OperationCost(0, 0); @@ -87,7 +88,7 @@ public final class OperationCost { /** * Cost of getFileStatus on root directory. */ - public static final OperationCost ROOT_FILE_STATUS_PROBE = NO_IO; + public static final OperationCost ROOT_FILE_STATUS_PROBE = NO_HEAD_OR_LIST; /** * Cost of {@link org.apache.hadoop.fs.s3a.impl.StatusProbeEnum#ALL}. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCostValidator.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCostValidator.java index 72e51ee3b9..3a8a1f6ad7 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCostValidator.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCostValidator.java @@ -20,6 +20,7 @@ import java.util.ArrayList; import java.util.Arrays; +import java.util.EnumSet; import java.util.List; import java.util.Map; import java.util.TreeMap; @@ -34,6 +35,7 @@ import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum; import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; import org.apache.hadoop.metrics2.lib.MutableCounter; import org.apache.hadoop.metrics2.lib.MutableMetric; @@ -274,7 +276,7 @@ public Builder withMetric(Statistic statistic) { /** * Add a varargs list of metrics. - * @param stat statistics to monitor. + * @param stats statistics to monitor. * @return this. */ public Builder withMetrics(Statistic...stats) { @@ -282,6 +284,20 @@ public Builder withMetrics(Statistic...stats) { return this; } + /** + * Add all counters and duration types to the + * metrics which can be asserted over. + * @return this. + */ + public Builder withAllCounters() { + EnumSet.allOf(Statistic.class).stream() + .filter(s -> + s.getType() == StatisticTypeEnum.TYPE_COUNTER + || s.getType() == StatisticTypeEnum.TYPE_DURATION) + .forEach(metrics::add); + return this; + } + /** * Instantiate. * @return the validator. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java index 946e59e9e3..91ea0c8e62 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java @@ -31,6 +31,7 @@ import org.apache.hadoop.fs.s3a.Statistic; import org.apache.hadoop.fs.s3a.WriteOperationHelper; import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.store.audit.AuditSpan; import org.apache.hadoop.util.functional.RemoteIterators; @@ -257,9 +258,9 @@ public void testMultiPagesListingPerformanceAndCorrectness() ObjectMetadata om = fs.newObjectMetadata(0L); PutObjectRequest put = requestFactory .newPutObjectRequest(fs.pathToKey(file), om, - new FailingInputStream()); + null, new FailingInputStream()); futures.add(submit(executorService, () -> - writeOperationHelper.putObject(put))); + writeOperationHelper.putObject(put, PutObjectOptions.keepingDirs()))); } LOG.info("Waiting for PUTs to complete"); waitForCompletion(futures); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AMultipartUploadSizeLimits.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AMultipartUploadSizeLimits.java index 231cfd884e..b83d12b4c1 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AMultipartUploadSizeLimits.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AMultipartUploadSizeLimits.java @@ -35,7 +35,7 @@ import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.Statistic; import org.apache.hadoop.fs.s3a.auth.ProgressCounter; -import org.apache.hadoop.fs.s3a.commit.CommitOperations; +import org.apache.hadoop.fs.s3a.commit.impl.CommitOperations; import static org.apache.hadoop.fs.StreamCapabilities.ABORTABLE_STREAM; import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;