Commit Graph

25084 Commits

Author SHA1 Message Date
PJ Fanning
d66dea300e
HADOOP-18341: upgrade commons-configuration2 to 2.8.0 and commons-text to 1.9 (#4916) 2022-09-22 10:44:27 +09:00
Ashutosh Gupta
683fa264ee
HADOOP-16769. LocalDirAllocator to provide diagnostics when file creation fails (#4896)
The patch provides detailed diagnostics of file creation failure in LocalDirAllocator.

Contributed by: Ashutosh Gupta
2022-09-21 11:54:47 +05:30
Ashutosh Gupta
3af155ceeb HADOOP-18400. Fix file split duplicating records from a succeeding split when reading BZip2 text files (#4732)
Co-authored-by: Ashutosh Gupta <ashugpt@amazon.com>
Signed-off-by: Akira Ajisaka <aajisaka@apache.org>
(cherry picked from commit 30c36ef25a)
2022-09-19 13:45:47 +09:00
Steve Vaughan
357c83db94
HDFS-16686. GetJournalEditServlet fails to authorize valid Kerberos request (#4724) (#4794) 2022-09-13 10:50:23 -07:00
Ashutosh Gupta
2532eca013
YARN-11241. Add uncleaning option for local app log file with log-aggregation enabled (#4703)
Co-authored-by: Ashutosh Gupta <ashugpt@amazon.com>
Signed-off-by: Akira Ajisaka <aajisaka@apache.org>
(cherry picked from commit 65a027b112)
2022-09-12 23:33:10 +09:00
Mukund Thakur
c9d6605a59 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 11:17:32 -05:00
Sumangala Patki
2e4c5ca88f
HADOOP-17873. ABFS: Fix transient failures in ITestAbfsStreamStatistics and ITestAbfsRestOperationException (#3699)
Successor for the reverted PR #3341, using the hadoop @VisibleForTesting attribute

Contributed by Sumangala Patki
2022-09-06 11:34:55 +01:00
sreeb-msft
5f3bc4340e
HADOOP-18408. ABFS: ITestAbfsManifestCommitProtocol fails on nonHNS configuration (#4758)
ITestAbfsManifestCommitProtocol  to set requireRenameResilience to false for nonHNS configuration

Contributed by Sree Bhattacharyya
2022-09-02 12:34:43 +01:00
monthonk
9dffa65021
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:15:48 +01:00
Steve Vaughan
3a6c8ff8bb
HDFS-16755. TestQJMWithFaults.testUnresolvableHostName() can fail due to unexpected host resolution (#4833)
Use ".invalid" domain from IETF RFC 2606 to ensure that the host doesn't resolve.

Contributed by Steve Vaughan Jr
2022-09-01 14:01:26 +01:00
Mukund Thakur
6cc5c92a89 HADOOP-18391. Improvements in VectoredReadUtils#readVectored() for direct buffers (#4787)
part of HADOOP-18103.

Contributed By: Mukund Thakur
2022-08-31 11:15:15 -05:00
Mukund Thakur
0a11ce2546 HADOOP-18407. Improve readVectored() api spec (#4760)
part of HADOOP-18103.

Contributed By: Mukund Thakur
2022-08-31 11:15:10 -05:00
Steve Loughran
f6c557d3b3
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 16:52:12 +01:00
Masatake Iwasaki
2a1701151c HADOOP-18375. Fix failure of shelltest for hadoop_add_ldlibpath. (#4652)
(cherry picked from commit 22835be63d)
2022-08-30 10:44:11 +00:00
Steve Vaughan
833fc64558
HDFS-16684. Exclude the current JournalNode (#4786)
The JournalNodeSyncer will include the local instance in syncing when using a bind host (e.g. 0.0.0.0).  There is a mechanism that is supposed to exclude the local instance, but it doesn't recognize the meta-address as a local address.

Running with bind addresses set to 0.0.0.0, the JournalNodeSyncer will log attempts to sync with itself as part of the normal syncing rotation.  For an HA configuration running 3 JournalNodes, the "other" list used by the JournalNodeSyncer will include 3 proxies.

Exclude bound local addresses, including the use of a wildcard address in the bound host configurations, while still allowing multiple instances on the same host.

Allow sync attempts with unresolved addresses, so that sync attempts can drive resolution as servers become available.

Backport.
Signed-off-by: stack <stack@apache.org>
2022-08-28 11:15:04 -07:00
zhengchenyu
3edddaf9fc HDFS-16732. [SBN READ] Avoid get location from observer when the block report is delayed (#4756)
Signed-off-by: Erik Krogen <xkrogen@apache.org>

(cherry picked from commit 231a4468cd)
2022-08-25 10:41:04 -07:00
Simba Dzinamarira
0326b7e935
HADOOP-18406: Adds alignment context to call path for creating RPC proxy with multiple connections per user.
Fixes #4748

Signed-off-by: Owen O'Malley <oomalley@linkedin.com>
2022-08-24 16:48:55 -07:00
xuzq
5b2d6684e6 HADOOP-13144. Enhancing IPC client throughput via multiple connections per user (#4542) 2022-08-24 16:48:35 -07:00
Ayush Saxena
9890a4aea4
Revert "HADOOP-18417. Upgrade to M7 of surefire plugin (#4795)"
This reverts commit 1ff121041c.
2022-08-25 03:53:34 +05:30
Steve Loughran
1168abc704
MAPREDUCE-7403. manifest-committer dynamic partitioning support. (#4728)
Declares its compatibility with Spark's dynamic
output partitioning by having the stream capability
"mapreduce.job.committer.dynamic.partitioning"

Requires a Spark release with SPARK-40034, which
does the probing before deciding whether to
accept/rejecting instantiation with
dynamic partition overwrite set

This feature can be declared as supported by
any other PathOutputCommitter implementations
whose algorithm and destination filesystem
are compatible.

None of the S3A committers are compatible.

The classic FileOutputCommitter is, but it
does not declare itself as such out of our fear
of changing that code. The Spark-side code
will automatically infer compatibility if
the created committer is of that class or
a subclass.

Contributed by Steve Loughran.
2022-08-24 11:19:05 +01:00
Steve Vaughan
98dd2b534f
HADOOP-18417. Upgrade to M7 of surefire plugin (#4795)
This addresses an issue where the plugin's default classpath
for executing tests fails to include
org.junit.platform.launcher.core.LauncherFactory.

Contributed by: Steve Vaughan Jr
2022-08-24 11:07:34 +01:00
Wei-Chiu Chuang
c4d94f5623
HADOOP-18333. Upgrade jetty version to 9.4.48.v20220622 (#4600)
* HADOOP-18001. Upgrade jetty version to 9.4.44 (#3700). Contributed by Yuan Luo.

Signed-off-by: Ayush Saxena <ayushsaxena@apache.org>
(cherry picked from commit b85c66a035)

* HADOOP-18333.Upgrade jetty version to 9.4.48.v20220622 (#4553)

Co-authored-by: Ashutosh Gupta <ashugpt@amazon.com>
(cherry picked from commit e664f81ce7)

 Conflicts:
	LICENSE-binary

Change-Id: I5a758df2551539c2780e170c3738c5b21eb0c79d

Co-authored-by: better3471 <46600375+better3471@users.noreply.github.com>
Co-authored-by: Ashutosh Gupta <ashutosh.gupta@st.niituniversity.in>
2022-08-24 08:16:49 +08:00
Simba Dzinamarira
e28dc524f6
HDFS-16669: Enhance client protocol to propagate last seen state IDs for multiple nameservices.
Fixes #4584

Signed-off-by: Owen O'Malley <oomalley@linkedin.com>
2022-08-23 11:27:21 -07:00
Steve Vaughan
4138661010
HDFS-16625. Check assumption about PMDK availability (#4788)
Co-authored-by: Steve Vaughan Jr <s_vaughan@apple.com>
2022-08-23 19:35:59 +09:00
Steve Vaughan
a73efb2d55
HDFS-16687. RouterFsckServlet replicates code from DfsServlet base class (#4681) (#4790) 2022-08-22 20:26:03 -07:00
Steve Vaughan
1120cc8485
HDFS-4043. Namenode Kerberos Login does not use proper hostname for host qualified hdfs principal name (#4785)
Use the existing DomainNameResolver to leverage the pluggable resolution framework.  This provides a means to perform a reverse lookup if needed.

Update default implementation of DNSDomainNameResolver to protect against returning the IP address as a string from a cached value.

Co-authored-by: Steve Vaughan Jr <s_vaughan@apple.com>
2022-08-23 05:34:33 +08:00
jianghuazhu
2123859d60
HDFS-16729. RBF: fix some unreasonably annotated docs. (#4745)
Reviewed-by: Inigo Goiri <inigoiri@apache.org>
Signed-off-by: Akira Ajisaka <aajisaka@apache.org>
(cherry picked from commit 7f176d080c)
2022-08-21 07:31:01 +09:00
Steve Vaughan
cfc11d2e5f
HADOOP-18365. Update the remote address when a change is detected (#4692) (#4768)
Back port to branch-3.3, to avoid reconnecting to the old address after detecting that the address has been updated.

* Use a stable hashCode to allow safe IP addr changes
* Add test that updated address is used

Once the address has been updated, it will be used in future calls.  Test verifies that a second request succeeds and that it uses the existing updated address instead of having to re-resolve.

Co-authored-by: Steve Vaughan Jr <s_vaughan@apple.com>
2022-08-19 18:56:02 -07:00
Viraj Jasani
51ddd02395 HADOOP-18403. Fix FileSystem leak in ITestS3AAWSCredentialsProvider (#4737)
Contributed By: Viraj Jasani
2022-08-18 17:45:44 -05:00
Ashutosh Gupta
a5d5d0708a
HADOOP-18385. ITestS3ACannedACLs failure; fixed by adding in a span (#4736)
Contributed by Ashutosh Gupta
2022-08-18 16:55:46 +01:00
Viraj Jasani
e8a28dc0d7 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 12:45:59 -05:00
Ashutosh Gupta
3b3bd89084 YARN-11248. Add unit test for FINISHED_CONTAINERS_PULLED_BY_AM event on DECOMMISSIONING (#4721)
Co-authored-by: Ashutosh Gupta <ashugpt@amazon.com>
Signed-off-by: Akira Ajisaka <aajisaka@apache.org>
(cherry picked from commit f02ff1afe2)
2022-08-16 19:07:42 +09:00
kevins-29
eff292bd5f
HADOOP-18383. Codecs with @DoNotPool annotation are not closed causing memory leak (#4739) 2022-08-15 10:14:02 -07:00
Steve Loughran
97763619c9
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 invoke 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:32:06 +01:00
Viraj Jasani
6b7c1329b2
HADOOP-18397. Shutdown AWSSecurityTokenService when its resources are no longer in use (#4722)
Contributed by Viraj Jasani.
2022-08-12 15:19:51 +01:00
Mukund Thakur
93c4704b33 HADOOP-18392. Propagate vectored s3a input stream stats to file system stats. (#4704)
part of HADOOP-18103.

Contributed By: Mukund Thakur
2022-08-11 15:24:25 -05:00
Mukund Thakur
09c8084191 HADOOP-18355. Update previous index properly while validating overlapping ranges. (#4647)
part of HADOOP-18103.

Contributed By: Mukund Thakur
2022-08-11 15:24:08 -05:00
Mukund Thakur
147a466c6d HADOOP-18227. Add input stream IOStats for vectored IO api in S3A. (#4636)
part of HADOOP-18103.

Contributed By: Mukund Thakur
2022-08-11 15:23:57 -05:00
huaxiangsun
1b9135e3b5
HADOOP-18340. deleteOnExit does not work with S3AFileSystem (#4608)
Contributed by Huaxiang Sun
2022-08-11 20:25:41 +01:00
Yubi Lee
a0e2ab2974
HADOOP-18398. Prevent AvroRecord*.class from being included non-test jar (#4727)
Contributed by Yubi Lee.
2022-08-11 20:16:52 +01:00
Viraj Jasani
0455769531
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 14:37:39 +01:00
Ashutosh Gupta
29ea8ceb49 HADOOP-18390. Fix out of sync import for HADOOP-18321 (#4694)
Co-authored-by: Ashutosh Gupta <ashugpt@amazon.com>
Signed-off-by: Akira Ajisaka <aajisaka@apache.org>
(cherry picked from commit bd0f9a46e1)
2022-08-07 16:06:09 +09:00
Ashutosh Gupta
3c339a11ec HADOOP-18321.Fix when to read an additional record from a BZip2 text file split (#4521)
* 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)
2022-08-06 21:53:48 +09:00
ahmarsuhail
351a9f732b
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:35 +01:00
Steve Loughran
9c5228cf6b
HADOOP-18305. Release Hadoop 3.3.4: upstream changelog and jdiff files
Add the r3.3.4 changelog, release notes and jdiff xml files.

Change-Id: I98b0fed54da3b810c3f23fe5b12e673937916257
2022-08-05 14:02:28 +01:00
xuzq
e024d1a3f8
HDFS-16712. Fix incorrect placeholder in DataNode.java (#4672). Contributed by ZanderXu.
Signed-off-by: Ayush Saxena <ayushsaxena@apache.org>
2022-08-03 13:02:42 +05:30
ahmarsuhail
4e842a7ff3 HADOOP-18368. Fixes ITestCustomSigner for access point names with '-' (#4634)
Contributed By: Ahmar Suhail <ahmarsu@amazon.co.uk>
2022-08-01 15:27:28 -05:00
Steve Loughran
7aebacef77 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:39:40 +01:00
ahmarsuhail
c653c58637
HADOOP-18372. ILoadTestS3ABulkDeleteThrottling failing. (#4642)
Contributed by Ahmar Suhail
2022-07-27 18:04:39 +01:00
Mehakmeet Singh
363f8138d2
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-27 11:23:06 +01:00