* 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.
(cherry picked from commit a432925f74)
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
Partial/Incomplete groups list can be returned in LDAP groups lookup.
Backported in #4550; minor tuning of parameters needed.
Contributed by larry mccay
Reduce the ExitUtil synchronized block scopes so System.exit
and Runtime.halt calls aren't within their boundaries,
so ExitUtil wrappers do not block each other.
Enlarged catches to all Throwables (not just Exceptions).
Contributed by Remi Catherinot
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.
Required for vectored IO feature. None of current buffer pool
implementation is complete. ElasticByteBufferPool doesn't use
weak references and could lead to memory leak errors and
DirectBufferPool doesn't support caller preferences of direct
and heap buffers and has only fixed length buffer implementation.
Contributed By: Mukund Thakur
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.
Regression caused by HDFS-16563; the hdfs exception text was changed, but because it was
a YARN test doing the check, Yetus didn't notice.
Contributed by zhengchenyu
* Add the changelog and release notes
* add all jdiff XML files
* update the project pom with the new stable version
Change-Id: Iaea846c3e451bbd446b45de146845a48953d580d
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
RBF proxy. There is a new configuration knob dfs.namenode.ip-proxy-users that configures
the list of users than can set their client ip address using the client context.
Fixes#4081
* 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