HADOOP-17833. Improve Magic Committer performance (#3289)
Speed up the magic committer with key changes being * Writes under __magic always retain directory markers * File creation under __magic skips all overwrite checks, including the LIST call intended to stop files being created over dirs. * mkdirs under __magic probes the path for existence but does not look any further. Extra parallelism in task and job commit directory scanning Use of createFile and openFile with parameters which all for HEAD checks to be skipped. The committer can write the summary _SUCCESS file to the path `fs.s3a.committer.summary.report.directory`, which can be in a different file system/bucket if desired, using the job id as the filename. Also: HADOOP-15460. S3A FS to add `fs.s3a.create.performance` Application code can set the createFile() option fs.s3a.create.performance to true to disable the same safety checks when writing under magic directories. Use with care. The createFile option prefix `fs.s3a.create.header.` can be used to add custom headers to S3 objects when created. Contributed by Steve Loughran.
This commit is contained in:
parent
020201cb65
commit
e199da3fae
@ -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}.
|
||||
*/
|
||||
|
@ -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<String> put(String key, String value) {
|
||||
if (value != null) {
|
||||
return evaluatedEntries.put(key, () -> value);
|
||||
} else {
|
||||
return evaluatedEntries.remove(key);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -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";
|
||||
|
@ -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));
|
||||
|
@ -34,7 +34,7 @@
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Unstable
|
||||
public final class CloseableTaskPoolSubmitter implements TaskPool.Submitter,
|
||||
public class CloseableTaskPoolSubmitter implements TaskPool.Submitter,
|
||||
Closeable {
|
||||
|
||||
/** Executors. */
|
||||
|
@ -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.
|
||||
|
||||
|
||||
## <a name="s3a"></a> 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.
|
@ -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<LocatedFileStatus> toList(
|
||||
RemoteIterator<LocatedFileStatus> iterator) throws IOException {
|
||||
ArrayList<LocatedFileStatus> list = new ArrayList<>();
|
||||
while (iterator.hasNext()) {
|
||||
list.add(iterator.next());
|
||||
}
|
||||
return list;
|
||||
return RemoteIterators.toList(iterator);
|
||||
}
|
||||
|
||||
/**
|
||||
@ -1464,11 +1462,7 @@ public static List<LocatedFileStatus> toList(
|
||||
*/
|
||||
public static <T extends FileStatus> List<T> iteratorToList(
|
||||
RemoteIterator<T> iterator) throws IOException {
|
||||
List<T> list = new ArrayList<>();
|
||||
while (iterator.hasNext()) {
|
||||
list.add(iterator.next());
|
||||
}
|
||||
return list;
|
||||
return RemoteIterators.toList(iterator);
|
||||
}
|
||||
|
||||
|
||||
|
@ -70,7 +70,6 @@ public static Path unmarshallPath(String path) {
|
||||
throw new RuntimeException(
|
||||
"Failed to parse \"" + path + "\" : " + e,
|
||||
e);
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -405,6 +405,39 @@
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-enforcer-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<id>banned-illegal-imports</id>
|
||||
<phase>process-sources</phase>
|
||||
<goals>
|
||||
<goal>enforce</goal>
|
||||
</goals>
|
||||
<configuration>
|
||||
<rules>
|
||||
<restrictImports>
|
||||
<includeTestCode>false</includeTestCode>
|
||||
<reason>Restrict mapreduce imports to committer code</reason>
|
||||
<exclusions>
|
||||
<exclusion>org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter</exclusion>
|
||||
<exclusion>org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitterFactory</exclusion>
|
||||
<exclusion>org.apache.hadoop.fs.s3a.commit.S3ACommitterFactory</exclusion>
|
||||
<exclusion>org.apache.hadoop.fs.s3a.commit.impl.*</exclusion>
|
||||
<exclusion>org.apache.hadoop.fs.s3a.commit.magic.*</exclusion>
|
||||
<exclusion>org.apache.hadoop.fs.s3a.commit.staging.*</exclusion>
|
||||
</exclusions>
|
||||
<bannedImports>
|
||||
<bannedImport>org.apache.hadoop.mapreduce.**</bannedImport>
|
||||
<bannedImport>org.apache.hadoop.mapred.**</bannedImport>
|
||||
</bannedImports>
|
||||
</restrictImports>
|
||||
</rules>
|
||||
</configuration>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
|
||||
|
@ -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";
|
||||
|
||||
}
|
||||
|
@ -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<PartETag> 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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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,34 +1635,37 @@ 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;
|
||||
EnumSet<CreateFlag> 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,
|
||||
// and attempting to do so can poison the load balancers with 404
|
||||
// entries.
|
||||
status = innerGetFileStatus(path, false,
|
||||
// just a directory (for safety)
|
||||
FileStatus status = innerGetFileStatus(path, false,
|
||||
overwrite
|
||||
? StatusProbeEnum.DIRECTORIES
|
||||
: StatusProbeEnum.ALL);
|
||||
@ -1668,15 +1681,22 @@ private FSDataOutputStream innerCreateFile(Path path,
|
||||
}
|
||||
LOG.debug("Overwriting file {}", path);
|
||||
} catch (FileNotFoundException e) {
|
||||
// this means the file is not found
|
||||
|
||||
// this means there is nothing at the path; all good.
|
||||
}
|
||||
}
|
||||
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.
|
||||
* <i>Important: this call will close any input stream in the request.</i>
|
||||
* @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,
|
||||
* <i>except under __magic</i> 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.
|
||||
* <p></p>
|
||||
* <p>
|
||||
* This operation MUST be called after any PUT/multipart PUT completes
|
||||
* successfully.
|
||||
* <p></p>
|
||||
* The actions include:
|
||||
* <ol>
|
||||
* <li>
|
||||
* Calling
|
||||
* <p>
|
||||
* The actions include calling
|
||||
* {@link #deleteUnnecessaryFakeDirectories(Path)}
|
||||
* if directory markers are not being retained.
|
||||
* </li>
|
||||
* <li>
|
||||
* Updating any metadata store with details on the newly created
|
||||
* object.
|
||||
* </li>
|
||||
* </ol>
|
||||
* @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);
|
||||
}
|
||||
|
@ -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();
|
||||
|
@ -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 <T> List<T> 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<LocatedFileStatus> listAndFilter(FileSystem fileSystem,
|
||||
public static RemoteIterator<LocatedFileStatus> 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()));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -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",
|
||||
|
@ -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<String, String> 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<PartETag> 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<PartETag> 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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -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> 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<String, String> 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<PartETag> 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
|
||||
*/
|
||||
|
@ -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.
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -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";
|
||||
|
||||
}
|
||||
|
@ -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() {
|
||||
}
|
||||
}
|
@ -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() {
|
||||
}
|
||||
|
||||
|
@ -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:
|
||||
* <ol>
|
||||
* <li>{@link #isMagicCommitPath(Path)} will always return false.</li>
|
||||
* <li>{@link #createTracker(Path, String)} will always return an instance
|
||||
* of {@link PutTracker}.</li>
|
||||
* <li>{@link #isUnderMagicPath(Path)} will always return false.</li>
|
||||
* <li>{@link #createTracker(Path, String, PutTrackerStatistics)} will always
|
||||
* return an instance of {@link PutTracker}.</li>
|
||||
* </ol>
|
||||
*
|
||||
* <p>Important</p>: must not directly or indirectly import a class which
|
||||
@ -88,9 +90,11 @@ public String keyOfFinalDestination(List<String> 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<String> 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<String> 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));
|
||||
}
|
||||
}
|
||||
|
@ -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 <I> item type being processed
|
||||
* @param <E> exception class which may be raised
|
||||
*/
|
||||
@FunctionalInterface
|
||||
public interface Task<I, E extends Exception> {
|
||||
void run(I item) throws E;
|
||||
}
|
||||
|
||||
/**
|
||||
* Callback invoked on a failure.
|
||||
* @param <I> item type being processed
|
||||
* @param <E> exception class which may be raised
|
||||
*/
|
||||
@FunctionalInterface
|
||||
public interface FailureTask<I, E extends Exception> {
|
||||
|
||||
/**
|
||||
* 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 <I> item type
|
||||
*/
|
||||
public static class Builder<I> {
|
||||
private final Iterable<I> items;
|
||||
private Submitter service = null;
|
||||
private FailureTask<I, ?> onFailure = null;
|
||||
private boolean stopOnFailure = false;
|
||||
private boolean suppressExceptions = false;
|
||||
private Task<I, ?> revertTask = null;
|
||||
private boolean stopRevertsOnFailure = false;
|
||||
private Task<I, ?> abortTask = null;
|
||||
private boolean stopAbortsOnFailure = false;
|
||||
|
||||
/**
|
||||
* Create the builder.
|
||||
* @param items items to process
|
||||
*/
|
||||
Builder(Iterable<I> 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<I> executeWith(Submitter submitter) {
|
||||
this.service = submitter;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder<I> onFailure(FailureTask<I, ?> task) {
|
||||
this.onFailure = task;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder<I> stopOnFailure() {
|
||||
this.stopOnFailure = true;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder<I> suppressExceptions() {
|
||||
return suppressExceptions(true);
|
||||
}
|
||||
|
||||
public Builder<I> suppressExceptions(boolean suppress) {
|
||||
this.suppressExceptions = suppress;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder<I> revertWith(Task<I, ?> task) {
|
||||
this.revertTask = task;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder<I> stopRevertsOnFailure() {
|
||||
this.stopRevertsOnFailure = true;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder<I> abortWith(Task<I, ?> task) {
|
||||
this.abortTask = task;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder<I> stopAbortsOnFailure() {
|
||||
this.stopAbortsOnFailure = true;
|
||||
return this;
|
||||
}
|
||||
|
||||
public <E extends Exception> boolean run(Task<I, E> task) throws E {
|
||||
if (service != null) {
|
||||
return runParallel(task);
|
||||
} else {
|
||||
return runSingleThreaded(task);
|
||||
}
|
||||
}
|
||||
|
||||
private <E extends Exception> boolean runSingleThreaded(Task<I, E> task)
|
||||
throws E {
|
||||
List<I> succeeded = new ArrayList<>();
|
||||
List<Exception> exceptions = new ArrayList<>();
|
||||
|
||||
Iterator<I> 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.<E>throwOne(exceptions);
|
||||
}
|
||||
|
||||
return !threw && exceptions.isEmpty();
|
||||
}
|
||||
|
||||
private <E extends Exception> boolean runParallel(final Task<I, E> task)
|
||||
throws E {
|
||||
final Queue<I> succeeded = new ConcurrentLinkedQueue<>();
|
||||
final Queue<Exception> exceptions = new ConcurrentLinkedQueue<>();
|
||||
final AtomicBoolean taskFailed = new AtomicBoolean(false);
|
||||
final AtomicBoolean abortFailed = new AtomicBoolean(false);
|
||||
final AtomicBoolean revertFailed = new AtomicBoolean(false);
|
||||
|
||||
List<Future<?>> 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.<E>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<Future<?>> 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 <I> Builder<I> foreach(Iterable<I> items) {
|
||||
return new Builder<>(items);
|
||||
}
|
||||
|
||||
public static <I> Builder<I> foreach(I[] items) {
|
||||
return new Builder<>(Arrays.asList(items));
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
private static <E extends Exception> void throwOne(
|
||||
Collection<Exception> exceptions)
|
||||
throws E {
|
||||
Iterator<Exception> iter = exceptions.iterator();
|
||||
Exception e = iter.next();
|
||||
Class<? extends Exception> exceptionClass = e.getClass();
|
||||
|
||||
while (iter.hasNext()) {
|
||||
Exception other = iter.next();
|
||||
if (!exceptionClass.isInstance(other)) {
|
||||
e.addSuppressed(other);
|
||||
}
|
||||
}
|
||||
|
||||
Tasks.<E>castAndThrow(e);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
private static <E extends Exception> 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);
|
||||
}
|
||||
|
||||
}
|
@ -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<PendingSet> {
|
||||
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<PendingSet> 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<PendingSet> 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<PendingSet> serializer) throws IOException {
|
||||
return saveFile(fs, path, this, serializer, true);
|
||||
}
|
||||
|
||||
/** @return the version marker. */
|
||||
|
@ -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<T extends 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<T> 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<T> 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 <T> 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 extends PersistentCommitData> T load(FileSystem fs,
|
||||
FileStatus status,
|
||||
JsonSerialization<T> 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 <T> 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 <T extends PersistentCommitData> IOStatistics saveFile(
|
||||
final FileSystem fs,
|
||||
final Path path,
|
||||
final T instance,
|
||||
final JsonSerialization<T> 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 <T> 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 <T extends PersistentCommitData> IOStatistics saveToStream(
|
||||
final Path path,
|
||||
final T instance,
|
||||
final FSDataOutputStreamBuilder builder,
|
||||
final JsonSerialization<T> 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();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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<String>, IOStatisticsSource {
|
||||
public class SinglePendingCommit extends PersistentCommitData<SinglePendingCommit>
|
||||
implements Iterable<String> {
|
||||
|
||||
/**
|
||||
* Serialization ID: {@value}.
|
||||
@ -141,26 +141,32 @@ public SinglePendingCommit() {
|
||||
* @return a serializer.
|
||||
*/
|
||||
public static JsonSerialization<SinglePendingCommit> 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<SinglePendingCommit> 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<SinglePendingCommit> serDeser,
|
||||
@Nullable FileStatus status)
|
||||
throws IOException {
|
||||
SinglePendingCommit instance = serializer().load(fs, path, status);
|
||||
JsonSerialization<SinglePendingCommit> 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<SinglePendingCommit> 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<SinglePendingCommit> serializer) throws IOException {
|
||||
return saveFile(fs, path, this, serializer, true);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -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 @@
|
||||
* <li>Not loadable? Something else.</li>
|
||||
* </ol>
|
||||
*
|
||||
* 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}
|
||||
* <p>
|
||||
* The JSON format SHOULD be considered public and evolving
|
||||
* with compatibility across versions.
|
||||
* <p>
|
||||
* 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.
|
||||
* <p>
|
||||
* 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<SuccessData> {
|
||||
|
||||
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<SuccessData> 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<SuccessData> 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<SuccessData> serializer() {
|
||||
return new JsonSerialization<>(SuccessData.class, false, true);
|
||||
public static JsonSerialization<SuccessData> 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);
|
||||
}
|
||||
}
|
||||
|
@ -29,12 +29,11 @@
|
||||
* <li>The summary information saved in the {@code _SUCCESS} file.</li>
|
||||
* </ol>
|
||||
*
|
||||
* 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
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
@ -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<JsonSerialization<PendingSet>>
|
||||
pendingSetSerializer =
|
||||
new WeakReferenceThreadMap<>((k) -> PendingSet.serializer(), null);
|
||||
|
||||
private final WeakReferenceThreadMap<JsonSerialization<SinglePendingCommit>>
|
||||
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<SinglePendingCommit> getSinglePendingFileSerializer() {
|
||||
return singleCommitSerializer.getForCurrentThread();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a serializer for .pendingset files.
|
||||
* @return a serializer.
|
||||
*/
|
||||
public JsonSerialization<PendingSet> 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();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
}
|
||||
}
|
@ -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);
|
||||
@ -227,8 +239,7 @@ private long innerCommit(
|
||||
commit.getDestinationKey(),
|
||||
commit.getUploadId(),
|
||||
toPartEtags(commit.getEtags()),
|
||||
commit.getLength()
|
||||
);
|
||||
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<LocatedFileStatus> locateAllSinglePendingCommits(
|
||||
public RemoteIterator<LocatedFileStatus> 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<PendingSet,
|
||||
List<Pair<LocatedFileStatus, IOException>>>
|
||||
loadSinglePendingCommits(Path pendingDir, boolean recursive)
|
||||
loadSinglePendingCommits(Path pendingDir,
|
||||
boolean recursive,
|
||||
CommitContext commitContext)
|
||||
throws IOException {
|
||||
|
||||
List<LocatedFileStatus> statusList = locateAllSinglePendingCommits(
|
||||
pendingDir, recursive);
|
||||
PendingSet commits = new PendingSet(
|
||||
statusList.size());
|
||||
List<Pair<LocatedFileStatus, IOException>> failures = new ArrayList<>(1);
|
||||
for (LocatedFileStatus status : statusList) {
|
||||
PendingSet commits = new PendingSet();
|
||||
List<SinglePendingCommit> pendingFiles = Collections.synchronizedList(
|
||||
new ArrayList<>(1));
|
||||
List<Pair<LocatedFileStatus, IOException>> failures = Collections.synchronizedList(
|
||||
new ArrayList<>(1));
|
||||
|
||||
TaskPool.foreach(locateAllSinglePendingCommits(pendingDir, recursive))
|
||||
//. stopOnFailure()
|
||||
.suppressExceptions(false)
|
||||
.executeWith(commitContext.getOuterSubmitter())
|
||||
.run(status -> {
|
||||
Path path = status.getPath();
|
||||
try {
|
||||
commits.add(SinglePendingCommit.load(fs, status.getPath(), status));
|
||||
// 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 {}", status.getPath(), 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());
|
||||
}
|
||||
}
|
||||
|
||||
@ -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<Long> 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) {
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
}
|
@ -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,7 +161,8 @@ public static Path getMagicTaskAttemptPath(TaskAttemptContext context,
|
||||
public static Path getBaseMagicTaskAttemptPath(TaskAttemptContext context,
|
||||
String jobUUID,
|
||||
Path dest) {
|
||||
return new Path(getMagicTaskAttemptsPath(jobUUID, dest),
|
||||
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 <i>not magic</i>
|
||||
* @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()));
|
||||
}
|
||||
|
@ -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;
|
@ -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<String, String> 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<String, String> 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
|
||||
|
@ -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,16 +196,18 @@ private PendingSet innerCommitTask(
|
||||
Path taskAttemptPath = getTaskAttemptPath(context);
|
||||
// load in all pending commits.
|
||||
CommitOperations actions = getCommitOperations();
|
||||
PendingSet pendingSet;
|
||||
try (CommitContext commitContext = initiateTaskOperation(context)) {
|
||||
Pair<PendingSet, List<Pair<LocatedFileStatus, IOException>>>
|
||||
loaded = actions.loadSinglePendingCommits(
|
||||
taskAttemptPath, true);
|
||||
PendingSet pendingSet = loaded.getKey();
|
||||
taskAttemptPath, true, commitContext);
|
||||
pendingSet = loaded.getKey();
|
||||
List<Pair<LocatedFileStatus, IOException>> 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);
|
||||
abortPendingUploads(commitContext, pendingSet.getCommits(), true);
|
||||
throw failures.get(0).getValue();
|
||||
}
|
||||
// patch in IDs
|
||||
@ -222,13 +229,16 @@ private PendingSet innerCommitTask(
|
||||
IOStatisticsLogging.demandStringifyIOStatisticsSource(pendingSet));
|
||||
try {
|
||||
// We will overwrite if there exists a pendingSet file already
|
||||
pendingSet.save(getDestFS(), taskOutcomePath, true);
|
||||
pendingSet.save(getDestFS(),
|
||||
taskOutcomePath,
|
||||
commitContext.getPendingSetSerializer());
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Failed to save task commit data to {} ",
|
||||
taskOutcomePath, e);
|
||||
abortPendingUploads(context, pendingSet.getCommits(), true);
|
||||
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());
|
||||
}
|
||||
|
||||
|
@ -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();
|
||||
|
@ -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<? extends FileStatus> taskOutput) throws IOException {
|
||||
List<? extends FileStatus> taskOutput,
|
||||
CommitContext commitContext) throws IOException {
|
||||
Path attemptPath = getTaskAttemptPath(context);
|
||||
Set<String> 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,
|
||||
* <li>APPEND: allowed.; no need to check.</li>
|
||||
* <li>REPLACE deletes all existing partitions.</li>
|
||||
* </ol>
|
||||
* @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(
|
||||
* }
|
||||
* </pre>
|
||||
*
|
||||
* @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<Path, String> 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);
|
||||
|
@ -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<LocatedFileStatus> 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<? extends FileStatus> taskOutput)
|
||||
List<? extends FileStatus> 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);
|
||||
}
|
||||
}
|
||||
|
@ -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<FSDataOutputStream, CreateFileBuilder> {
|
||||
|
||||
/**
|
||||
* Flag set to create with overwrite.
|
||||
*/
|
||||
public static final EnumSet<CreateFlag> CREATE_OVERWRITE_FLAGS =
|
||||
EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE);
|
||||
|
||||
/**
|
||||
* Flag set to create without overwrite.
|
||||
*/
|
||||
public static final EnumSet<CreateFlag> 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<String, String> headers = new HashMap<>();
|
||||
final Set<String> mandatoryKeys = getMandatoryKeys();
|
||||
final Set<String> 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<CreateFlag> 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<CreateFlag> 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<CreateFlag> 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<CreateFlag> flags;
|
||||
|
||||
/**
|
||||
* create parent dirs?
|
||||
*/
|
||||
private final boolean recursive;
|
||||
|
||||
/**
|
||||
* performance flag.
|
||||
*/
|
||||
private final boolean performance;
|
||||
|
||||
/**
|
||||
* Headers; may be null.
|
||||
*/
|
||||
private final Map<String, String> headers;
|
||||
|
||||
/**
|
||||
* @param flags creation flags
|
||||
* @param recursive create parent dirs?
|
||||
* @param performance performance flag
|
||||
* @param headers nullable header map.
|
||||
*/
|
||||
public CreateFileOptions(
|
||||
final EnumSet<CreateFlag> flags,
|
||||
final boolean recursive,
|
||||
final boolean performance,
|
||||
final Map<String, String> 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<CreateFlag> getFlags() {
|
||||
return flags;
|
||||
}
|
||||
|
||||
public boolean isRecursive() {
|
||||
return recursive;
|
||||
}
|
||||
|
||||
public boolean isPerformance() {
|
||||
return performance;
|
||||
}
|
||||
|
||||
public Map<String, String> getHeaders() {
|
||||
return headers;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -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<String> CREATE_FILE_KEYS =
|
||||
Collections.unmodifiableSet(
|
||||
new HashSet<>(Arrays.asList(Constants.FS_S3A_CREATE_PERFORMANCE)));
|
||||
|
||||
}
|
||||
|
@ -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
|
||||
* <ul>
|
||||
* <li>The only path check is for a directory already existing there.</li>
|
||||
* <li>No ancestors are checked</li>
|
||||
* <li>Parent markers are never deleted, irrespective of FS settings</li>
|
||||
* </ul>
|
||||
* As a result, irrespective of depth, the operations performed are only
|
||||
* <ol>
|
||||
* <li>One LIST</li>
|
||||
* <li>If needed, one PUT</li>
|
||||
* </ol>
|
||||
*/
|
||||
public class MkdirOperation extends ExecutingStoreOperation<Boolean> {
|
||||
|
||||
@ -51,13 +63,21 @@ public class MkdirOperation extends ExecutingStoreOperation<Boolean> {
|
||||
|
||||
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;
|
||||
}
|
||||
}
|
||||
|
@ -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<String, String> 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<String, String> 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<String, String> 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;
|
||||
}
|
||||
|
||||
}
|
@ -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<String, String> headers = options.getHeaders();
|
||||
if (headers != null) {
|
||||
objectMetadata.setUserMetadata(headers);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a builder.
|
||||
* @return new builder.
|
||||
|
@ -123,7 +123,8 @@ public CompletableFuture<UploadHandle> 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)));
|
||||
|
@ -25,7 +25,8 @@
|
||||
* Block output stream statistics.
|
||||
*/
|
||||
public interface BlockOutputStreamStatistics extends Closeable,
|
||||
S3AStatisticInterface {
|
||||
S3AStatisticInterface,
|
||||
PutTrackerStatistics {
|
||||
|
||||
/**
|
||||
* Block is queued for upload.
|
||||
|
@ -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();
|
||||
}
|
||||
|
@ -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 {
|
||||
}
|
@ -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
|
||||
|
@ -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())
|
||||
|
@ -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.
|
||||
|
||||
<property>
|
||||
<name>fs.s3a.committer.threads</name>
|
||||
<value>8</value>
|
||||
<value>-4</value>
|
||||
<description>
|
||||
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.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
|
@ -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 {
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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();
|
||||
|
@ -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";
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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
|
||||
|
@ -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,24 +596,25 @@ 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<PendingSet, List<Pair<LocatedFileStatus, IOException>>>
|
||||
pendingCommits = operations.loadSinglePendingCommits(readOnlyDir,
|
||||
true);
|
||||
true, commitContext);
|
||||
|
||||
// all those commits must fail
|
||||
List<SinglePendingCommit> commits = pendingCommits.getLeft().getCommits();
|
||||
assertEquals(range, commits.size());
|
||||
try(CommitOperations.CommitContext commitContext
|
||||
= operations.initiateCommitOperation(uploadDest)) {
|
||||
commits.parallelStream().forEach(
|
||||
(c) -> {
|
||||
CommitOperations.MaybeIOE maybeIOE =
|
||||
@ -620,12 +622,11 @@ public void testRestrictedCommitActions() throws Throwable {
|
||||
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,
|
||||
|
@ -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<String> 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;
|
||||
}
|
||||
|
||||
|
@ -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<JobData> abortInTeardown = new ArrayList<>(1);
|
||||
private final List<JobData> 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());
|
||||
}
|
||||
}
|
||||
|
@ -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<String> listMultipartUploads(
|
||||
String prefix) throws IOException {
|
||||
|
||||
return MultipartTestUtils.listMultipartUploads(getFileSystem(), prefix);
|
||||
}
|
||||
|
||||
}
|
@ -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;
|
||||
* <ol>
|
||||
* <li>Even on marker deleting filesystems,
|
||||
* operations under magic dirs do not trigger marker deletion.</li>
|
||||
* <li>Loading pending files from FileStatus entries skips HEAD checks.</li>
|
||||
* <li>Mkdir under magic dirs doesn't check ancestor or dest type</li>
|
||||
* </ol>
|
||||
*/
|
||||
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<LocatedFileStatus> 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));
|
||||
}
|
||||
|
||||
}
|
@ -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<String> elements = splitPathToElements(pendingPath);
|
||||
assertEquals("splitPathToElements()", filename, lastElement(elements));
|
||||
Assertions.assertThat(lastElement(elements))
|
||||
.describedAs("splitPathToElements(%s)", pendingPath)
|
||||
.isEqualTo(filename);
|
||||
List<String> 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)
|
||||
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<String> etags = persisted.getEtags();
|
||||
assertEquals("etag list " + persisted, 1, etags.size());
|
||||
List<PartETag> 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");
|
||||
|
||||
Path dest = methodSubPath("testUploadMissingFile");
|
||||
intercept(FileNotFoundException.class, () ->
|
||||
actions.uploadFileToPendingCommit(tempFile, dest, null,
|
||||
DEFAULT_MULTIPART_SIZE, progress);
|
||||
progress.assertCount("Progress counter should be 1.",
|
||||
1);
|
||||
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<SinglePendingCommit> 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();
|
||||
|
@ -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);
|
||||
|
@ -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<Object[]> params() {
|
||||
return Arrays.asList(new Object[][]{
|
||||
{0},
|
||||
{1},
|
||||
{3},
|
||||
{8},
|
||||
{16},
|
||||
});
|
||||
}
|
||||
|
||||
|
||||
private List<Item> 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<Item> builder() {
|
||||
return Tasks.foreach(items).executeWith(submitter);
|
||||
}
|
||||
|
||||
private void assertRun(Tasks.Builder<Item> builder,
|
||||
CounterTask task) throws IOException {
|
||||
boolean b = builder.run(task);
|
||||
assertTrue("Run of " + task + " failed", b);
|
||||
}
|
||||
|
||||
private void assertFailed(Tasks.Builder<Item> 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<Function<Item, Boolean>> 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<Item, Boolean> 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<Item, IOException> {
|
||||
|
||||
private CounterTask(String name, int limit,
|
||||
Function<Item, Boolean> action) {
|
||||
super(name, limit, action);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run(Item item) throws IOException {
|
||||
process(item);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private final class FailureCounter
|
||||
extends BaseCounter implements Tasks.FailureTask<Item, IOException> {
|
||||
private Exception exception;
|
||||
|
||||
private FailureCounter(String name, int limit,
|
||||
Function<Item, Boolean> 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;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
@ -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<LocatedFileStatus> filtered = listAndFilter(fs,
|
||||
List<LocatedFileStatus> 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 + '/');
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -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<String> 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);
|
||||
try(CommitContext commitContext
|
||||
= operations.createCommitContextForTesting(jobDir, null, COMMITTER_THREADS)) {
|
||||
Pair<PendingSet, List<Pair<LocatedFileStatus, IOException>>>
|
||||
results = operations.loadSinglePendingCommits(jobDir, false);
|
||||
try(CommitOperations.CommitContext commitContext
|
||||
= operations.initiateCommitOperation(jobDir)) {
|
||||
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();
|
||||
|
@ -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<String> filenames,
|
||||
final IOStatisticsSnapshot ioStatistics)
|
||||
throws IOException {
|
||||
//skipped
|
||||
return null;
|
||||
}
|
||||
|
||||
public ClientResults getResults() throws IOException {
|
||||
|
@ -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.
|
||||
*/
|
||||
|
@ -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<SinglePendingCommit> 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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<SinglePendingCommit> 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<String> runTasks(JobContext jobContext,
|
||||
TaskAttemptContext attempt = new TaskAttemptContextImpl(
|
||||
new Configuration(jobContext.getConfiguration()), attemptID);
|
||||
MockedStagingCommitter taskCommitter = new MockedStagingCommitter(
|
||||
outputPath, attempt);
|
||||
getOutputPath(), attempt);
|
||||
commitTask(taskCommitter, attempt, numFiles);
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
@ -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"));
|
||||
|
@ -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);
|
||||
|
@ -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.
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
@ -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<String, String> 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<String, String> 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();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -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));
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -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));
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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}.
|
||||
|
@ -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.
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
Loading…
Reference in New Issue
Block a user