Improve task commit resilience everywhere
and add an option to reduce delete IO requests on
job cleanup (relevant for ABFS and HDFS).
Task Commit Resilience
----------------------
Task manifest saving is re-attempted on failure; the number of
attempts made is configurable with the option:
mapreduce.manifest.committer.manifest.save.attempts
* The default is 5.
* The minimum is 1; asking for less is ignored.
* A retry policy adds 500ms of sleep per attempt.
* Move from classic rename() to commitFile() to rename the file,
after calling getFileStatus() to get its length and possibly etag.
This becomes a rename() on gcs/hdfs anyway, but on abfs it does reach
the ResilientCommitByRename callbacks in abfs, which report on
the outcome to the caller...which is then logged at WARN.
* New statistic task_stage_save_summary_file to distinguish from
other saving operations (job success/report file).
This is only saved to the manifest on task commit retries, and
provides statistics on all previous unsuccessful attempts to save
the manifests
+ test changes to match the codepath changes, including improvements
in fault injection.
Directory size for deletion
---------------------------
New option
mapreduce.manifest.committer.cleanup.parallel.delete.base.first
This attempts an initial attempt at deleting the base dir, only falling
back to parallel deletes if there's a timeout.
This option is disabled by default; Consider enabling it for abfs to
reduce IO load. Consult the documentation for more details.
Success file printing
---------------------
The command to print a JSON _SUCCESS file from this committer and
any S3A committer is now something which can be invoked from
the mapred command:
mapred successfile <path to file>
Contributed by Steve Loughran
This adds borad support for Amazon S3 Express One Zone to the S3A connector,
particularly resilience of other parts of the codebase to LIST operations returning
paths under which only in-progress uploads are taking place.
hadoop-common and hadoop-mapreduce treewalking routines all cope with this;
distcp is left alone.
There are still some outstanding followup issues, and we expect more to surface
with extended use.
Contains HADOOP-18955. AWS SDK v2: add path capability probe "fs.s3a.capability.aws.v2
* lets us probe for AWS SDK version
* bucket-info reports it
Contains HADOOP-18961 S3A: add s3guard command "bucket"
hadoop s3guard bucket -create -region us-west-2 -zone usw2-az2 \
s3a://stevel--usw2-az2--x-s3/
* requires -zone if bucket is zonal
* rejects it if not
* rejects zonal bucket suffixes if endpoint is not aws (safety feature)
* imperfect, but a functional starting point.
New path capability "fs.s3a.capability.zonal.storage"
* Used in tests to determine whether pending uploads manifest paths
* cli tests can probe for this
* bucket-info reports it
* some tests disable/change assertions as appropriate
----
Shell commands fail on S3Express buckets if pending uploads.
New path capability in hadoop-common
"fs.capability.directory.listing.inconsistent"
1. S3AFS returns true on a S3 Express bucket
2. FileUtil.maybeIgnoreMissingDirectory(fs, path, fnfe)
decides whether to swallow the exception or not.
3. This is used in: Shell, FileInputFormat, LocatedFileStatusFetcher
Fixes with tests
* fs -ls -R
* fs -du
* fs -df
* fs -find
* S3AFS.getContentSummary() (maybe...should discuss)
* mapred LocatedFileStatusFetcher
* Globber, HADOOP-15478 already fixed that when dealing with
S3 inconsistencies
* FileInputFormat
S3Express CreateSession request is permitted outside audit spans.
S3 Bulk Delete calls request the store to return the list of deleted objects
if RequestFactoryImpl is set to trace.
log4j.logger.org.apache.hadoop.fs.s3a.impl.RequestFactoryImpl=TRACE
Test Changes
* ITestS3AMiscOperations removes all tests which require unencrypted
buckets. AWS S3 defaults to SSE-S3 everywhere.
* ITestBucketTool to test new tool without actually creating new
buckets.
* S3ATestUtils add methods to skip test suites/cases if store is/is not
S3Express
* Cutting down on "is this a S3Express bucket" logic to trailing --x-s3 string
and not worrying about AZ naming logic. commented out relevant tests.
* ITestTreewalkProblems validated against standard and S3Express stores
Outstanding
* Distcp: tests show it fails. Proposed: release notes.
---
x-amz-checksum header not found when signing S3Express messages
This modifies the custom signer in ITestCustomSigner to be a subclass
of AwsS3V4Signer with a goal of preventing signing problems with
S3 Express stores.
----
RemoteFileChanged renaming multipart file
Maps 412 status code to RemoteFileChangedException
Modifies huge file tests
-Adds a check on etag match for stat vs list
-ITestS3AHugeFilesByteBufferBlocks renames parent dirs, rather than
files, to replicate distcp better.
----
S3Express custom Signing cannot handle bulk delete
Copy custom signer into production JAR, so enable downstream testing
Extend ITestCustomSigner to cover more filesystem operations
- PUT
- POST
- COPY
- LIST
- Bulk delete through delete() and rename()
- list + abort multipart uploads
Suite is parameterized on bulk delete enabled/disabled.
To use the new signer for a full test run:
<property>
<name>fs.s3a.custom.signers</name>
<value>CustomSdkSigner:org.apache.hadoop.fs.s3a.auth.CustomSdkSigner</value>
</property>
<property>
<name>fs.s3a.s3.signing-algorithm</name>
<value>CustomSdkSigner</value>
</property>
In container-log4j.properties, log4j.appender.{APPENDER}.MaxFileSize is set to ${yarn.app.container.log.filesize}, but yarn.app.container.log.filesize is 0 in default. So log is missing. This log is always rolling and only show the latest log.
By default, the mapreduce manifest committer is used for jobs working with abfs and gcs.
Hadoop mapreduce will pick this up automatically; for Spark it is a bit complicated: read the docs
to see the steps required.
This modifies the manifest committer so that the list of files
to rename is passed between stages as a file of
writeable entries on the local filesystem.
The map of directories to create is still passed in memory;
this map is built across all tasks, so even if many tasks
created files, if they all write into the same set of directories
the memory needed is O(directories) with the
task count not a factor.
The _SUCCESS file reports on heap size through gauges.
This should give a warning if there are problems.
Contributed by Steve Loughran
This:
1. Adds optLong, optDouble, mustLong and mustDouble
methods to the FSBuilder interface to let callers explicitly
passin long and double arguments.
2. The opt() and must() builder calls which take float/double values
now only set long values instead, so as to avoid problems
related to overloaded methods resulting in a ".0" being appended
to a long value.
3. All of the relevant opt/must calls in the hadoop codebase move to
the new methods
4. And the s3a code is resilient to parse errors in is numeric options
-it will downgrade to the default.
This is nominally incompatible, but the floating-point builder methods
were never used: nothing currently expects floating point numbers.
For anyone who wants to safely set numeric builder options across all compatible
releases, convert the number to a string and then use the opt(String, String)
and must(String, String) methods.
Contributed by Steve Loughran
Declares its compatibility with Spark's dynamic
output partitioning by having the stream capability
"mapreduce.job.committer.dynamic.partitioning"
Requires a Spark release with SPARK-40034, which
does the probing before deciding whether to
accept/rejecting instantiation with
dynamic partition overwrite set
This feature can be declared as supported by
any other PathOutputCommitter implementations
whose algorithm and destination filesystem
are compatible.
None of the S3A committers are compatible.
The classic FileOutputCommitter is, but it
does not declare itself as such out of our fear
of changing that code. The Spark-side code
will automatically infer compatibility if
the created committer is of that class or
a subclass.
Contributed by Steve Loughran.
* HADOOP-18321.Fix when to read an additional record from a BZip2 text file split
Co-authored-by: Ashutosh Gupta <ashugpt@amazon.com> and Reviewed by Akira Ajisaka.
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.