Removed JUnit APIs from WebServicesTestUtils and TestContainerLogsUtils.
They are used by MapReduce modules as well as YARN modules, so the
APIs need to be removed to upgrade the JUnit version on a per-module basis.
Also, this effectively reverts the prior fix in #5209 because it didn't actually
fix the issue.
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.
This downgrades jackson from the version switched to in
HADOOP-18033 (2.13.0), to Jackson 2.12.7.
This removes the dependency on javax.ws.rs-api,
so avoiding runtime problems with applications using
jersey-core v1 and/or jsr311-api.
The 2.12.7 release still contains the fix for CVE-2020-36518.
Contributed by PJ Fanning
* 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.
These changes ensure that sequential files are opened with the
right read policy, and split start/end is passed in.
As well as offering opportunities for filesystem clients to
choose fetch/cache/seek policies, the settings ensure that
processing text files on an s3 bucket where the default policy
is "random" will still be processed efficiently.
This commit depends on the associated hadoop-common patch,
which must be committed first.
Contributed by Steve Loughran.
Change-Id: Ic6713fd752441cf42ebe8739d05c2293a5db9f94