Commit Graph

552 Commits

Author SHA1 Message Date
Viraj Jasani
648071e197
HADOOP-18466. Limit the findbugs suppression IS2_INCONSISTENT_SYNC to S3AFileSystem field (#4926)
Follow-on to HADOOP-18455.

Contributed by Viraj Jasani
2022-09-26 18:56:58 +01:00
Viraj Jasani
084b68e380
HADOOP-18455. S3A prefetching executor should be closed (#4879)
follow-on patch to HADOOP-18186. 

Contributed by: Viraj Jasani
2022-09-22 00:22:41 +05:30
Viraj Jasani
5b1657278c
HADOOP-18377. hadoop-aws build to add a -prefetch profile to run all tests with prefetching (#4914)
Contributed by Viraj Jasani
2022-09-20 10:26:13 +01:00
Mukund Thakur
8732625f50
HADOOP-18439. Fix VectoredIO for LocalFileSystem when checksum is enabled. (#4862)
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
2022-09-09 21:46:08 +05:30
Viraj Jasani
56387cce57
HADOOP-18186. s3a prefetching to use SemaphoredDelegatingExecutor for submitting work (#4796)
Contributed by Viraj Jasani
2022-09-09 11:32:20 +01:00
Mehakmeet Singh
03961b10c2
HADOOP-18416. fix ITestS3AIOStatisticsContext test failure (#4806)
Follow on to HADOOP-17461.

Contributed by: Mehakmeet Singh
2022-09-08 21:03:18 +05:30
monthonk
20560401ec
HADOOP-18339. S3A storage class option only picked up when buffering writes to disk. (#4669)
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
2022-09-01 18:14:32 +01:00
Mukund Thakur
19830c98bc
HADOOP-18391. Improvements in VectoredReadUtils#readVectored() for direct buffers (#4787)
part of HADOOP-18103.

Contributed By: Mukund Thakur
2022-08-31 21:41:41 +05:30
Steve Loughran
c69e16b297
HADOOP-18410. S3AInputStream.unbuffer() does not release http connections (#4766)
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.
2022-08-31 11:16:52 +01:00
ahmarsuhail
7fb9c306e2
HADOOP-18382. AWS SDK v2 upgrade prerequisites (#4698)
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
2022-08-25 17:36:48 +01:00
Viraj Jasani
c249db80c2
HADOOP-18380. fs.s3a.prefetch.block.size to be read through longBytesOption (#4762)
Contributed by Viraj Jasani.
2022-08-23 10:49:04 +01:00
Viraj Jasani
7f030250b4
HADOOP-18403. Fix FileSystem leak in ITestS3AAWSCredentialsProvider (#4737)
Contributed By: Viraj Jasani
2022-08-19 04:14:43 +05:30
Ashutosh Gupta
d09dd4a0b9
HADOOP-18385. ITestS3ACannedACLs failure; fixed by adding in a span (#4736)
Contributed by Ashutosh Gupta
2022-08-18 13:57:43 +01:00
Steve Loughran
682931a6ac
HADOOP-18028. High performance S3A input stream (#4752)
This is the the preview release of the HADOOP-18028 S3A performance input stream.
It is still stabilizing, but ready to test.

Contains

HADOOP-18028. High performance S3A input stream (#4109)
	Contributed by Bhalchandra Pandit.

HADOOP-18180. Replace use of twitter util-core with java futures (#4115)
	Contributed by PJ Fanning.

HADOOP-18177. Document prefetching architecture. (#4205)
	Contributed by Ahmar Suhail

HADOOP-18175. fix test failures with prefetching s3a input stream (#4212)
 Contributed by Monthon Klongklaew

HADOOP-18231.  S3A prefetching: fix failing tests & drain stream async.  (#4386)

	* adds in new test for prefetching input stream
	* creates streamStats before opening stream
	* updates numBlocks calculation method
	* fixes ITestS3AOpenCost.testOpenFileLongerLength
	* drains stream async
	* fixes failing unit test

	Contributed by Ahmar Suhail

HADOOP-18254. Disable S3A prefetching by default. (#4469)
	Contributed by Ahmar Suhail

HADOOP-18190. Collect IOStatistics during S3A prefetching (#4458)

	This adds iOStatisticsConnection to the S3PrefetchingInputStream class, with
	new statistic names in StreamStatistics.

	This stream is not (yet) IOStatisticsContext aware.

	Contributed by Ahmar Suhail

HADOOP-18379 rebase feature/HADOOP-18028-s3a-prefetch to trunk
HADOOP-18187. Convert s3a prefetching to use JavaDoc for fields and enums.
HADOOP-18318. Update class names to be clear they belong to S3A prefetching
	Contributed by Steve Loughran
2022-08-18 13:53:06 +01:00
Viraj Jasani
d55d76e1e2
HADOOP-18371. S3A FS init to log at debug when fs.s3a.create.storage.class is unset (#4730)
Contributed By: Viraj Jasani
2022-08-16 03:55:58 +05:30
Steve Loughran
906ae5138e
HADOOP-18402. S3A committer NPE in spark job abort (#4735)
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 invok abortJob() with an
incomplete TaskContext.

This patch MUST be applied to branches containing
HADOOP-17833. "Improve Magic Committer Performance."

Contributed by Steve Loughran.
2022-08-15 11:20:59 +01:00
Steve Loughran
eee59a8372
Revert "HADOOP-18402. S3A committer NPE in spark job abort (#4735)"
(managed to commit through the github ui before I'd got the message done)

This reverts commit ad83e95046.
2022-08-15 11:20:36 +01:00
Steve Loughran
ad83e95046
HADOOP-18402. S3A committer NPE in spark job abort (#4735)
jobId.toString() to only be called when the ID isn't null.

this doesn't surface in MR, but spark seems to manage it

Change-Id: I06692ef30a4af510c660d7222292932a8d4b5147
2022-08-15 11:18:47 +01:00
Viraj Jasani
8c9533a0f8
HADOOP-18397. Shutdown AWSSecurityTokenService when its resources are no longer in use (#4722)
Contributed by Viraj Jasani.
2022-08-12 11:59:15 +01:00
Mukund Thakur
b28e4c6904
HADOOP-18392. Propagate vectored s3a input stream stats to file system stats. (#4704)
part of HADOOP-18103.

Contributed By: Mukund Thakur
2022-08-12 01:42:00 +05:30
huaxiangsun
e9509ac467
HADOOP-18340. deleteOnExit does not work with S3AFileSystem (#4608)
Contributed by Huaxiang Sun
2022-08-11 20:25:13 +01:00
Viraj Jasani
06f0f7db79
HADOOP-18373. IOStatisticsContext tuning (#4705)
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
2022-08-08 10:42:57 +01:00
ahmarsuhail
b5642c5638
HADOOP-18366. ITestS3Select.testSelectSeekFullLandsat is timing out. (#4702)
Reduces size of data read to 1 MB

Contributed by Ahmar Suhail
2022-08-05 14:13:04 +01:00
ahmarsuhail
123d1aa884
HADOOP-18368. Fixes ITestCustomSigner for access point names with '-' (#4634)
Contributed By: Ahmar Suhail <ahmarsu@amazon.co.uk>
2022-08-02 01:49:42 +05:30
Mukund Thakur
a5b12c8010
HADOOP-18227. Add input stream IOStats for vectored IO api in S3A. (#4636)
part of HADOOP-18103.

Contributed By: Mukund Thakur
2022-07-28 21:57:37 +05:30
Steve Loughran
58ed621304
HADOOP-18344. Upgrade AWS SDK to 1.12.262 (#4637)
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
2022-07-28 11:29:38 +01:00
ahmarsuhail
c92ff0b4f1
HADOOP-18372. ILoadTestS3ABulkDeleteThrottling failing. (#4642)
Contributed by Ahmar Suhail
2022-07-27 17:19:57 +01:00
Mehakmeet Singh
4c8cd61961
HADOOP-17461. Collect thread-level IOStatistics. (#4352)
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
2022-07-26 20:41:22 +01:00
ashutoshpant
bac2219e3c
HADOOP-18330. S3AFileSystem removes Path when calling createS3Client (#4572)
Adds a new parameter object in s3ClientCreationParameters that holds 
the full s3a path URI

Contributed by Ashutosh Pant
2022-07-21 10:16:39 +01:00
Mukund Thakur
4d1f6f9b99 HADOOP-18106: Handle memory fragmentation in S3A Vectored IO. (#4445)
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
2022-06-22 17:29:32 +01:00
Mukund Thakur
1408dd89a7 HADOOP-18107 Adding scale test for vectored reads for large file (#4273)
part of HADOOP-18103.

Contributed By: Mukund Thakur
2022-06-22 17:29:32 +01:00
Mukund Thakur
5db0f34e29 HADOOP-18104: S3A: Add configs to configure minSeekForVectorReads and maxReadSizeForVectorReads (#3964)
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
2022-06-22 17:29:32 +01:00
Mukund Thakur
2daf0a814f HADOOP-11867. Add a high-performance vectored read API. (#3904)
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
2022-06-22 17:29:32 +01:00
Steve Loughran
e199da3fae
HADOOP-17833. Improve Magic Committer performance (#3289)
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.
2022-06-17 19:11:35 +01:00
monthonk
5ac55b405d
HADOOP-12020. Add s3a storage class option fs.s3a.create.storage.class (#3877)
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
2022-06-08 19:05:17 +01:00
Ashutosh Gupta
a46ef5f2eb
HADOOP-18234. Fix s3a access point xml examples (#4309)
Contributed by Ashutosh Gupta
2022-05-16 17:47:14 +01:00
Daniel Carl Jones
4230162a76
HADOOP-18168. Fix S3A ITestMarkerTool use of purged public bucket. (#4140)
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
2022-05-03 14:28:08 +01:00
Steve Loughran
6ec39d45c9 Revert "HADOOP-18168. . (#4140)"
This reverts commit 6ab7b72cd6.
2022-05-03 14:27:52 +01:00
Daniel Carl Jones
6ab7b72cd6
HADOOP-18168. . (#4140)
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
2022-05-03 14:26:52 +01:00
Steve Loughran
e0cd0a82e0
HADOOP-16202. Enhanced openFile(): hadoop-aws changes. (#2584/3)
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
2022-04-24 17:33:05 +01:00
Daniel Carl Jones
a6ebc42671
HADOOP-18201. Remove endpoint config overrides for ITestS3ARequesterPays (#4169)
Contributed by Daniel Carl Jones.
2022-04-14 16:21:34 +01:00
Daniel Carl Jones
9edfe30a60
HADOOP-14661. Add S3 requester pays bucket support to S3A (#3962)
Adds the option fs.s3a.requester.pays.enabled, which, if set to true, allows
the client to access S3 buckets where the requester is billed for the IO.

Contributed by Daniel Carl Jones
2022-03-23 20:00:50 +00:00
Steve Loughran
708a0ce21b
HADOOP-13704. Optimized S3A getContentSummary()
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
2022-03-22 13:21:12 +00:00
Mukund Thakur
672e380c4f
HADOOP-18112: Implement paging during multi object delete. (#4045)
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
2022-03-11 13:05:45 +05:30
Viraj Jasani
66b72406bd
HADOOP-18131. Upgrade maven enforcer plugin and relevant dependencies (#4000)
Reviewed-by: Akira Ajisaka <aajisaka@apache.org>
Reviewed-by: Wei-Chiu Chuang <weichiu@apache.org>
Signed-off-by: Takanobu Asanuma <tasanuma@apache.org>
2022-03-08 17:27:04 +09:00
Mehakmeet Singh
6995374b54
HADOOP-18150. Fix ITestAuditManagerDisabled test in S3A. (#4044)
Contributed by Mehakmeet Singh
2022-03-03 18:44:28 +00:00
monthonk
1f157f802d
HADOOP-17386. Change default fs.s3a.buffer.dir to be under Yarn container path on yarn applications (#3908)
Co-authored-by: Monthon Klongklaew <monthonk@amazon.com>
Signed-off-by: Akira Ajisaka <aajisaka@apache.org>
2022-02-22 13:50:27 +09:00
Steve Loughran
efdec92cab
HADOOP-18091. S3A auditing leaks memory through ThreadLocal references (#3930)
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.
2022-02-10 12:31:41 +00:00
Joey Krabacher
a08e69d33e
HADOOP-18114. Documentation correction in assumed_roles.md (#3949)
Fixes typo in hadoop-aws/assumed_roles.md

Contributed by Joey Krabacher
2022-02-09 10:35:11 +00:00
Petre Bogdan Stolojan
5e7ce26e66
HADOOP-18085. S3 SDK Upgrade causes AccessPoint ARN endpoint mistranslation (#3902)
Part of HADOOP-17198. Support S3 Access Points.

HADOOP-18068. "upgrade AWS SDK to 1.12.132" broke the access point endpoint
translation.

Correct endpoints should start with "s3-accesspoint.", after SDK upgrade they start with
"s3.accesspoint-" which messes up tests + region detection by the SDK.

Contributed by Bogdan Stolojan
2022-02-04 15:37:08 +00:00