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 defines standard option and values for the
openFile() builder API for opening a file:
fs.option.openfile.read.policy
A list of the desired read policy, in preferred order.
standard values are
adaptive, default, random, sequential, vector, whole-file
fs.option.openfile.length
How long the file is.
fs.option.openfile.split.start
start of a task's split
fs.option.openfile.split.end
end of a task's split
These can be used by filesystem connectors to optimize their
reading of the source file, including but not limited to
* skipping existence/length probes when opening a file
* choosing a policy for prefetching/caching data
The hadoop shell commands which read files all declare "whole-file"
and "sequential", as appropriate.
Contributed by Steve Loughran.
Change-Id: Ia290f79ea7973ce8713d4f90f1315b24d7a23da1
To get the new behavior, define fs.viewfs.trash.force-inside-mount-point to be true.
If the trash root for path p is in the same mount point as path p,
and one of:
* The mount point isn't at the top of the target fs.
* The resolved path of path is root (eg it is the fallback FS).
* The trash root isn't in user's target fs home directory.
get the corresponding viewFS path for the trash root and return it.
Otherwise, use <mnt>/.Trash/<user>.
Signed-off-by: Owen O'Malley <oomalley@linkedin.com>
(cherry picked from commit 8b8158f02d)
Co-authored-by: Xing Lin <xinglin@linkedin.com>
Optimize the scan for s3 by performing a deep tree listing,
inferring directory counts from the paths returned.
Contributed by Ahmar Suhail.
Change-Id: I26ffa8c6f65fd11c68a88d6e2243b0eac6ffd024
* New statistic names in StoreStatisticNames
(for joint use with s3a committers)
* Improvements to IOStatistics implementation classes
* RateLimiting wrapper to guava RateLimiter
* S3A committer Tasks moved over as TaskPool and
added support for RemoteIterator
* JsonSerialization.load() to fail fast if source does not exist
+ tests.
This commit is a prerequisite for the main MAPREDUCE-7341 Manifest Committer
patch.
Contributed by Steve Loughran
Change-Id: Ia92e2ab5083ac3d8d3d713a4d9cb3e9e0278f654
(cherry picked from commit d0fa9b5775)
Conflicts:
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java
Co-authored-by: Gautham B A <gautham.bangalore@gmail.com>
Multi object delete of size more than 1000 is not supported by S3 and
fails with MalformedXML error. So implementing paging of requests to
reduce the number of keys in a single request. Page size can be configured
using "fs.s3a.bulk.delete.page.size"
Contributed By: Mukund Thakur
Adds a new map type WeakReferenceMap, which stores weak
references to values, and a WeakReferenceThreadMap subclass
to more closely resemble a thread local type, as it is a
map of threadId to value.
Construct it with a factory method and optional callback
for notification on loss and regeneration.
WeakReferenceThreadMap<WrappingAuditSpan> activeSpan =
new WeakReferenceThreadMap<>(
(k) -> getUnbondedSpan(),
this::noteSpanReferenceLost);
This is used in ActiveAuditManagerS3A for span tracking.
Relates to
* HADOOP-17511. Add an Audit plugin point for S3A
* HADOOP-18094. Disable S3A auditing by default.
Contributed by Steve Loughran.
Change-Id: Ibf7bb082fd47298f7ebf46d92f56e80ca9b2aaf8
This (big!) patch adds support for client side encryption in AWS S3,
with keys managed by AWS-KMS.
Read the documentation in encryption.md very, very carefully before
use and consider it unstable.
S3-CSE is enabled in the existing configuration option
"fs.s3a.server-side-encryption-algorithm":
fs.s3a.server-side-encryption-algorithm=CSE-KMS
fs.s3a.server-side-encryption.key=<KMS_KEY_ID>
You cannot enable CSE and SSE in the same client, although
you can still enable a default SSE option in the S3 console.
* Filesystem list/get status operations subtract 16 bytes from the length
of all files >= 16 bytes long to compensate for the padding which CSE
adds.
* The SDK always warns about the specific algorithm chosen being
deprecated. It is critical to use this algorithm for ranged
GET requests to work (i.e. random IO). Ignore.
* Unencrypted files CANNOT BE READ.
The entire bucket SHOULD be encrypted with S3-CSE.
* Uploading files may be a bit slower as blocks are now
written sequentially.
* The Multipart Upload API is disabled when S3-CSE is active.
Contributed by Mehakmeet Singh
Change-Id: Ie1a27a036a39db66a67e9c6d33bc78d54ea708a0
Addresses the problem of processes running out of memory when
there are many ABFS output streams queuing data to upload,
especially when the network upload bandwidth is less than the rate
data is generated.
ABFS Output streams now buffer their blocks of data to
"disk", "bytebuffer" or "array", as set in
"fs.azure.data.blocks.buffer"
When buffering via disk, the location for temporary storage
is set in "fs.azure.buffer.dir"
For safe scaling: use "disk" (default); for performance, when
confident that upload bandwidth will never be a bottleneck,
experiment with the memory options.
The number of blocks a single stream can have queued for uploading
is set in "fs.azure.block.upload.active.blocks".
The default value is 20.
Contributed by Mehakmeet Singh.
This adds a new class org.apache.hadoop.util.Preconditions which is
* @Private/@Unstable
* Intended to allow us to move off Google Guava
* Is designed to be trivially backportable
(i.e contains no references to guava classes internally)
Please use this instead of the guava equivalents, where possible.
Contributed by: Ahmed Hussein
Change-Id: Ic392451bcfe7d446184b7c995734bcca8c07286e