This is to try and close the underlying filesystems when the FileContext APIs are used.
Without this, threads may be leaked
Contributed by Steve Loughran
The AWS SDKV2 upgrade log no longer warns about instantiation
of the v1 SDK credential providers which are commonly used in
s3a configurations:
* com.amazonaws.auth.EnvironmentVariableCredentialsProvider
* com.amazonaws.auth.EC2ContainerCredentialsProviderWrapper
* com.amazonaws.auth.InstanceProfileCredentialsProvider
When the hadoop-aws module moves to the v2 SDK, references to these
credential providers will be rewritten to their v2 equivalents.
Follow-on to HADOOP-18382. "Upgrade AWS SDK to V2 - Prerequisites"
Contributed by Ahmar Suhail
This patch prepares the hadoop-aws module for a future
migration to using the v2 AWS SDK (HADOOP-18073)
That upgrade will be incompatible; this patch prepares
for it:
-marks some credential providers and other
classes and methods as @deprecated.
-updates site documentation
-reduces the visibility of the s3 client;
other than for testing, it is kept private to
the S3AFileSystem class.
-logs some warnings when deprecated APIs are used.
The warning messages are printed only once
per JVM's life. To disable them, set the
log level of org.apache.hadoop.fs.s3a.SDKV2Upgrade
to ERROR
Contributed by Ahmar Suhail
The swift:// connector for openstack support has been removed.
The hadoop-openstack jar remains, only now it is empty of code.
This is to ensure that projects which declare the JAR a dependency
will still have successful builds.
Contributed by Steve Loughran
Add to XMLUtils a set of methods to create secure XML Parsers/transformers,
locking down DTD, schema, XXE exposure.
Use these wherever XML parsers are created.
Contributed by PJ Fanning
part of HADOOP-18103.
Also introducing a config fs.s3a.vectored.active.ranged.reads
to configure the maximum number of number of range reads a
single input stream can have active (downloading, or queued)
to the central FileSystem instance's pool of queued operations.
This stops a single stream overloading the shared thread pool.
Contributed by: Mukund Thakur
Conflicts:
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
part of HADOOP-18103.
While merging the ranges in CheckSumFs, they are rounded up based on the
value of checksum bytes size which leads to some ranges crossing the EOF
thus they need to be fixed else it will cause EOFException during actual reads.
Contributed By: Mukund Thakur
Follow-up to HADOOP-12020 Support configuration of different S3 storage classes;
S3 storage class is now set when buffering to heap/bytebuffers, and when
creating directory markers
Contributed by Monthon Klongklaew
HADOOP-16202 "Enhance openFile()" added asynchronous draining of the
remaining bytes of an S3 HTTP input stream for those operations
(unbuffer, seek) where it could avoid blocking the active
thread.
This patch fixes the asynchronous stream draining to work and so
return the stream back to the http pool. Without this, whenever
unbuffer() or seek() was called on a stream and an asynchronous
drain triggered, the connection was not returned; eventually
the pool would be empty and subsequent S3 requests would
fail with the message "Timeout waiting for connection from pool"
The root cause was that even though the fields passed in to drain() were
converted to references through the methods, in the lambda expression
passed in to submit, they were direct references
operation = client.submit(
() -> drain(uri, streamStatistics,
false, reason, remaining,
object, wrappedStream)); /* here */
Those fields were only read during the async execution, at which
point they would have been set to null (or even a subsequent read).
A new SDKStreamDrainer class peforms the draining; this is a Callable
and can be submitted directly to the executor pool.
The class is used in both the classic and prefetching s3a input streams.
Also, calling unbuffer() switches the S3AInputStream from adaptive
to random IO mode; that is, it is considered a cue that future
IO will not be sequential, whole-file reads.
Contributed by Steve Loughran.
JobID.toString() and TaskID.toString() to only be called
when the IDs are not null.
This doesn't surface in MapReduce, but Spark SQL can trigger
in job abort, where it may invoke abortJob() with an
incomplete TaskContext.
This patch MUST be applied to branches containing
HADOOP-17833. "Improve Magic Committer Performance."
Contributed by Steve Loughran.
The name of the option to enable/disable thread level statistics is
"fs.iostatistics.thread.level.enabled";
There is also an enabled() probe in IOStatisticsContext which can
be used to see if the thread level statistics is active.
Contributed by Viraj Jasani
Fixes CVE-2018-7489 in shaded jackson.
+Add more commands in testing.md
to the CLI tests needed when qualifying
a release
Contributed by Steve Loughran
This adds a thread-level collector of IOStatistics, IOStatisticsContext,
which can be:
* Retrieved for a thread and cached for access from other
threads.
* reset() to record new statistics.
* Queried for live statistics through the
IOStatisticsSource.getIOStatistics() method.
* Queries for a statistics aggregator for use in instrumented
classes.
* Asked to create a serializable copy in snapshot()
The goal is to make it possible for applications with multiple
threads performing different work items simultaneously
to be able to collect statistics on the individual threads,
and so generate aggregate reports on the total work performed
for a specific job, query or similar unit of work.
Some changes in IOStatistics-gathering classes are needed for
this feature
* Caching the active context's aggregator in the object's
constructor
* Updating it in close()
Slightly more work is needed in multithreaded code,
such as the S3A committers, which collect statistics across
all threads used in task and job commit operations.
Currently the IOStatisticsContext-aware classes are:
* The S3A input stream, output stream and list iterators.
* RawLocalFileSystem's input and output streams.
* The S3A committers.
* The TaskPool class in hadoop-common, which propagates
the active context into scheduled worker threads.
Collection of statistics in the IOStatisticsContext
is disabled process-wide by default until the feature
is considered stable.
To enable the collection, set the option
fs.thread.level.iostatistics.enabled
to "true" in core-site.xml;
Contributed by Mehakmeet Singh and Steve Loughran
HADOOP-18313: AliyunOSSBlockOutputStream should not mark the temporary file for deletion. Contributed by wujinhu.
(cherry picked from commit 3ec4b932c1)
ABFS rename fails intermittently when the Storage-blob tracking
metadata is in an incomplete state. This surfaces as the error code
404 and an error message of "RenameDestinationParentPathNotFound"
To mitigate this issue, when a request fails with this response.
the ABFS client issues a HEAD call on the source file
and then retries the rename operation again
ABFS filesystem statistics track when this occurs with new counters
rename_recovery
metadata_incomplete_rename_failures
rename_path_attempts
This is very rare occurrence and appears to be triggered under certain
heavy load conditions, just as with HADOOP-18163.
Contributed by Mehakmeet Singh.
part of HADOOP-18103.
Handling memory fragmentation in S3A vectored IO implementation by
allocating smaller user range requested size buffers and directly
filling them from the remote S3 stream and skipping undesired
data in between ranges.
This patch also adds aborting active vectored reads when stream is
closed or unbuffer() is called.
Contributed By: Mukund Thakur
Conflicts:
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
Part of HADOOP-18103.
Introducing fs.s3a.vectored.read.min.seek.size and fs.s3a.vectored.read.max.merged.size
to configure min seek and max read during a vectored IO operation in S3A connector.
These properties actually define how the ranges will be merged. To completely
disable merging set fs.s3a.max.readsize.vectored.read to 0.
Contributed By: Mukund Thakur
part of HADOOP-18103.
Add support for multiple ranged vectored read api in PositionedReadable.
The default iterates through the ranges to read each synchronously,
but the intent is that FSDataInputStream subclasses can make more
efficient readers especially in object stores implementation.
Also added implementation in S3A where smaller ranges are merged and
sliced byte buffers are returned to the readers. All the merged ranged are
fetched from S3 asynchronously.
Contributed By: Owen O'Malley and Mukund Thakur
Conflicts:
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
pom.xml
Updating the hadoop version of branch-3.3 to 3.3.9-SNAPSHOT
pending agreement on what number its future release should take.
Using 3.3.9-SNAPSHOT puts space in for other incremental releases,
while avoiding creating JIRA release ordering and autocompletion
confusion the way adding a 3.3.10 or higher version would do.
Contributed by Steve Loughran
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.
Adds a new option fs.s3a.create.storage.class which can
be used to set the storage class for files created in AWS S3.
Consult the documentation for details and instructions on how
disable the relevant tests when testing against third-party
stores.
Contributed by Monthon Klongklaew
Change-Id: I8cdebadf294a89fde08d98729ad96f251d58411c
This moves off use of the purged s3a://landsat-pds bucket, so fixing tests
which had started failing.
* Adds a new class, PublicDatasetTestUtils to manage the use of public datasets.
* The new test bucket s3a://usgs-landsat/ is requester pays, so depends upon
HADOOP-14661.
Consult the updated test documentation when running against other S3 stores.
Contributed by Daniel Carl Jones
Change-Id: Ie8585e4d9b67667f8cb80b2970225d79a4f8d257
Stops the abfs connector warning if openFile().withFileStatus()
is invoked with a FileStatus is not an abfs VersionedFileStatus.
Contributed by Steve Loughran.
Change-Id: I85076b365eb30aaef2ed35139fa8714efd4d048e
S3A input stream support for the few fs.option.openfile settings.
As well as supporting the read policy option and values,
if the file length is declared in fs.option.openfile.length
then no HEAD request will be issued when opening a file.
This can cut a few tens of milliseconds off the operation.
The patch adds a new openfile parameter/FS configuration option
fs.s3a.input.async.drain.threshold (default: 16000).
It declares the number of bytes remaining in the http input stream
above which any operation to read and discard the rest of the stream,
"draining", is executed asynchronously.
This asynchronous draining offers some performance benefit on seek-heavy
file IO.
Contributed by Steve Loughran.
Change-Id: I9b0626bbe635e9fd97ac0f463f5e7167e0111e39
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