Commit Graph

572 Commits

Author SHA1 Message Date
Ankit Saurabh 22f6d55b71
HADOOP-18246. Reduce lower limit on fs.s3a.prefetch.block.size to 1 byte. (#5120)
The minimum value of fs.s3a.prefetch.block.size is now 1

Contributed by Ankit Saurabh
2023-02-02 18:45:21 +00:00
Nikita Eshkeev 4de31123ce
Fix "the the" and friends typos (#5267)
Signed-off-by: Nikita Eshkeev <neshkeev@yandex.ru>
2023-01-17 03:33:59 +08:00
ahmarsuhail 9c6eeb699e
HADOOP-18320. Fixes typos in Delegation Tokens documentation. (#4499)
Contributed By: Ahmar Suhail
2023-01-09 22:18:41 +05:30
Steve Loughran aaf92fe183
HADOOP-18526. Leak of S3AInstrumentation instances via hadoop Metrics references (#5144)
This has triggered an OOM in a process which was churning through s3a fs
instances; the increased memory footprint of IOStatistics amplified what
must have been a long-standing issue with FS instances being created
and not closed()

*  Makes sure instrumentation is closed when the FS is closed.
*  Uses a weak reference from metrics to instrumentation, so even
   if the FS wasn't closed (see HADOOP-18478), this back reference
   would not cause the S3AInstrumentation reference to be retained.
*  If S3AFileSystem is configured to log at TRACE it will log the
   calling stack of initialize(), so help identify where the
   instance is being created. This should help track down
   the cause of instance leakage.

Contributed by Steve Loughran.
2022-12-14 18:21:03 +00:00
Steve Loughran 1cecf8ab70
HADOOP-18183. s3a audit logs to publish range start/end of GET requests. (#5110)
The start and end of the range is set in a new audit param "rg",
e.g "?rg=100-200"

Contributed by Ankit Saurabh
2022-12-14 14:01:28 +00:00
Oleksandr Shevchenko 0a4528cd7f
HADOOP-18563. Misleading AWS SDK S3 timeout configuration comment (#5197)
Contributed by Oleksandr Shevchenko
2022-12-08 15:07:59 +00:00
Ashutosh Gupta 2c1158e858
HADOOP-18531. Fix assertion failure in ITestS3APrefetchingInputStream (#5149)
This patch MUST be applied to all branches containing HADOOP-18378
so as to ensure reliable test runs.

Contributed by Ashutosh Gupta
2022-11-23 17:47:39 +00:00
Daniel Carl Jones 0b577992ef
HADOOP-18482. ITestS3APrefetchingInputStream to skip if CSV test file unavailable (#4983)
Contributed by Danny Jones
2022-10-31 21:19:34 +00:00
sabertiger af7dd660e0
HADOOP-18233. Possible race condition with TemporaryAWSCredentialsProvider (#5024)
This fixes a race condition with the TemporaryAWSCredentialProvider
one which has existed for a long time but which only surfaced
(usually in Spark) when the bucket existence probe was disabled
by setting fs.s3a.bucket.probe to 0 -a performance speedup
which was made the default in HADOOP-17454.

Contributed by Jimmy Wong.
2022-10-31 12:43:30 +00:00
Mehakmeet Singh fba46aa5bb
HADOOP-18499. S3A to support HTTPS web proxies (#5051)
The option "fs.s3a.proxy.ssl.enabled" controls
whether the s3a connects to a proxy over HTTP (default) or HTTPS.
Set to "true" to use HTTPS.

Contributed by Mehakmeet Singh
2022-10-26 11:45:20 +01:00
Viraj Jasani 8aa04b0b24
HADOOP-18189 S3APrefetchingInputStream to support status probes when closed (#5036)
Contributed by Viraj Jasani
2022-10-19 14:38:11 +01:00
Daniel Carl Jones 6207ac47e0
HADOOP-18304. Improve user-facing S3A committers documentation (#4478)
Contributed by: Daniel Carl Jones
2022-10-19 12:56:47 +05:30
Steve Loughran d80db6c9e5
HADOOP-18476. Abfs and S3A FileContext bindings to close wrapped filesystems in finalizer (#4966)
This is to try and close the underlying filesystems when the FileContext APIs are used.
Without this, threads may be leaked
2022-10-18 14:53:02 +01:00
Ankit Saurabh 2d91daab5e
HADOOP-18156. Address JavaDoc warnings in classes like MarkerTool, S3ObjectAttributes, etc (#4965)
Contributed by Ankit Saurabh
2022-10-17 18:10:47 +01:00
ahmarsuhail 77e551a478
HADOOP-18481. AWS v2 SDK upgrade log to not about standard AWS Credential Providers. (#4973)
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
2022-10-14 10:48:09 +01:00
monthonk 9439d8e4e4
HADOOP-18292. Fix s3 select tests when running against unsupported storage class (#4489)
Follow-on from HADOOP-12020.

Contributed by Monthon Klongklaew
2022-10-13 13:13:36 +01:00
Mukund Thakur be70bbb4be
HADOOP-18460. checkIfVectoredIOStopped before populating the buffers (#4986)
Contributed by Mukund Thakur
2022-10-10 11:17:45 +01:00
Daniel Carl Jones 7ec762a5fd
HADOOP-18465. Fix S3A SSE test skip when encryption is disabled (#4925)
Contributed by Daniel Carl Jones
2022-10-06 12:42:01 +01:00
Alessandro Passaro 1675a28e5a
HADOOP-18378. Implement lazy seek in S3A prefetching. (#4955)
Make S3APrefetchingInputStream.seek() completely lazy. Calls to seek() will not affect the current buffer nor interfere with prefetching, until read() is called.

This change allows various usage patterns to benefit from prefetching, e.g. when calling readFully(position, buffer) in a loop for contiguous positions the intermediate internal calls to seek() will be noops and prefetching will have the same performance as in a sequential read.

Contributed by Alessandro Passaro.
2022-10-06 12:00:41 +01:00
Mukund Thakur 735e35d648
HADOOP-18347. S3A Vectored IO to use bounded thread pool. (#4918)
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
2022-09-27 21:13:07 +05:30
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