Skip to content

Conversation

@steveloughran
Copy link
Contributor

HADOOP-19569.

Executors in hadoop-common to

  • pick up shutdown of inner executor and shut themselves down.
  • semaphore executor to decrement counters in this process so that queue state is updated This stops callers being able to submit work when the inner executor has shut down.

S3A code

  • StoreImpl to IllegalStateException on method invocation whene the service isn't running. Some methods are kept open as they do seem needed.
  • WriteOperationHelper callbacks to raise IllegalStateException when invoked when FS is closed.

This is complex.

TODO:

  • WriteOperationHelper MUST make all calls to the FS through its callback interface, rather than given a ref to S3AFS. This makes it easy to identify and lock down the methods.
  • What is the correct exception to raise in write/close() failures? IOE or illegal state?

How was this patch tested?

New ITests which close the FS while simple and multipart writes are in progress.

S3 london.

For code changes:

  • Does the title or this PR starts with the corresponding JIRA issue id (e.g. 'HADOOP-17799. Your PR title ...')?
  • Object storage: have the integration tests been executed and the endpoint declared according to the connector-specific documentation?
  • If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under ASF 2.0?
  • If applicable, have you updated the LICENSE, LICENSE-binary, NOTICE-binary files?

@steveloughran steveloughran marked this pull request as draft May 21, 2025 10:39
@steveloughran steveloughran force-pushed the s3/HADOOP-19569-stream-write-fs-close branch from cf32b54 to fea3e89 Compare May 27, 2025 11:17
@steveloughran steveloughran requested a review from Copilot May 30, 2025 10:48
@apache apache deleted a comment from hadoop-yetus May 30, 2025
@apache apache deleted a comment from hadoop-yetus May 30, 2025
Copy link

Copilot AI left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Pull Request Overview

This PR addresses HADOOP-19569 by enhancing the S3A filesystem’s behavior when closed, ensuring that write/close operations fail gracefully, and improves internal service registration and executor shutdown behavior. Key changes include enforcing FS state checks via added checkRunning() calls, refactoring helper and callback methods (e.g. renaming getWriteOperationHelper to createWriteOperationHelperWithinActiveSpan and replacing direct FS calls with getStore() invocations), and cleaning up legacy utilities such as MultipartUtils.

Reviewed Changes

Copilot reviewed 32 out of 32 changed files in this pull request and generated 2 comments.

Show a summary per file
File Description
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java Adjusted WriteOperationHelper construction with updated callback parameters and method renaming.
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClientSideEncryption.java
Updated test calls to use getStore() for invoking putObjectDirect.
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java
Inserted checkRunning() calls across API methods and updated service lookup/register methods.
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/WriteOperationHelper.java Refactored to use a new “callbacks” field and improved null validations with requireNonNull.
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java Exposed getStore() publicly and updated multipart upload handling to leverage the new MultipartIOService.
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestBlockingThreadPoolExecutorService.java
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SemaphoredDelegatingExecutor.java
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/BlockingThreadPoolExecutorService.java
Enhanced executor service tests and added rejection-handling logic to ensure proper shutdown on task rejection.
Comments suppressed due to low confidence (3)

hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/WriteOperationHelper.java:127

  • [nitpick] Update the Javadoc in WriteOperationHelper to reflect the change from using the variable name 'writeOperationHelperCallbacks' to 'callbacks', ensuring that the documentation clearly explains its role and usage.
private final WriteOperationHelperCallbacks callbacks;

hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java:290

  • The completeMultipartUpload method has been removed from the S3AStore interface. Ensure that all external consumers have been updated accordingly or consider adding a deprecation warning to ease the transition.
/* Removed completeMultipartUpload method */

hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java:1357

  • [nitpick] Making the getStore() method public exposes internal implementation details. Confirm that this exposure is intended, or consider providing a more restricted accessor if external access is not required.
public S3AStore getStore() {

slower than enqueueing. */
final BlockingQueue<Runnable> workQueue =
new LinkedBlockingQueue<>(waitingTasks + activeTasks);
final InnerExecutorRejection rejection = new InnerExecutorRejection();
Copy link

Copilot AI May 30, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

[nitpick] The InnerExecutorRejection handler now shuts down the service upon rejection. Consider enhancing the error handling logic or adding more detailed documentation to explain the shutdown behavior in case of task rejection.

Copilot uses AI. Check for mistakes.
private static final Integer SOME_VALUE = 1337;

private static BlockingThreadPoolExecutorService tpe;
private BlockingThreadPoolExecutorService tpe;
Copy link

Copilot AI May 30, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

[nitpick] Since the thread pool executor is now an instance variable with setup/teardown methods, ensure that each test properly initializes and destroys the executor to avoid interference between tests.

Copilot uses AI. Check for mistakes.
@steveloughran steveloughran force-pushed the s3/HADOOP-19569-stream-write-fs-close branch from c347def to c8cdbfa Compare June 18, 2025 12:24
@steveloughran steveloughran force-pushed the s3/HADOOP-19569-stream-write-fs-close branch 2 times, most recently from 1dee57d to b66a4e9 Compare July 1, 2025 09:59
@steveloughran steveloughran force-pushed the s3/HADOOP-19569-stream-write-fs-close branch from b66a4e9 to e90a8af Compare July 9, 2025 16:35
@steveloughran steveloughran marked this pull request as ready for review July 10, 2025 12:20
@steveloughran steveloughran force-pushed the s3/HADOOP-19569-stream-write-fs-close branch from e90a8af to 6fceeb3 Compare July 10, 2025 12:30
@steveloughran steveloughran force-pushed the s3/HADOOP-19569-stream-write-fs-close branch from 6fceeb3 to 1038637 Compare September 19, 2025 12:59
@steveloughran steveloughran force-pushed the s3/HADOOP-19569-stream-write-fs-close branch from a20587d to 4a80578 Compare November 13, 2025 18:09
@steveloughran steveloughran force-pushed the s3/HADOOP-19569-stream-write-fs-close branch from 4a80578 to b7fc4d2 Compare December 8, 2025 14:57
All upload operations are in MultipartIO service, which
has been renamed and move to package org.apache.hadoop.fs.s3a.impl.write
to match.

For completeness deletion should also go into this class or an adjacent one on
deletion.

Pulled out multipart IO such that
there are no back references from it to
S3AStore -the final change is to define a store
statistics class which it and other things can
use to update stats.

Executors in hadoop-common to
 - pick up shutdown of inner executor and shut themselves down.
 - semaphore executor to decrement counters in this process so that
   queue state is updated
 - semaphored delegating executor unit test in common
This stops callers being able to submit work when the inner executor has
shut down.

WriteOperationHelper
* make all calls through its callback interface, rather than given a ref to S3AFS.
* Move WriteOperationHelper callbacks to S3Store layer,

Multipart IO operations
* move nearly all Multpart IO operationss out of s3afs and into a
  new mulitpart service interface and impl
* Multipart service retrieved and invoked as appropriate
* StoreImpl stores a map of ServiceName -> service.
  with a lookupService() method in S3AStore interface, it's possible to
  retrieve services through the API just by knowing their name and type
* registering all current services this way

StoreImpl to IllegalStateException on  method invocation whene the service
isn't running.

Some methods are kept open as they do seem needed.
@steveloughran steveloughran force-pushed the s3/HADOOP-19569-stream-write-fs-close branch from b7fc4d2 to 2290412 Compare December 30, 2025 18:22
@steveloughran
Copy link
Contributor Author

rebased. Tested against rustfs on localhost out of curiousity, so now I know what to say if someone asks if it can be used: "not if you need consistent directory listings"

  1. Eventually consistently path listing on delete.

Looking into the docs https://deepwiki.com/rustfs/rustfs/5.5-concurrency-management-and-caching#cache-coherence-and-invalidation comes up with "Invalidation is performed asynchronously to avoid blocking the write path."

This is not as bad as AWS S3 used to be where a 404 was cached after a failing HEAD request, and listings took time to even find a newly created object.

[ERROR] org.apache.hadoop.fs.contract.s3a.ITestS3AContractRootDir.testListEmptyRootDirectory -- Time elapsed: 0.059 s <<< FAILURE!
org.opentest4j.AssertionFailedError: Deleted file: unexpectedly found s3a://rustybucket/job-00 as  S3AFileStatus{path=s3a://rustybucket/job-00; isDirectory=true; modification_time=0; access_time=0; owner=stevel; group=stevel; permission=rwxrwxrwx; isSymlink=false; hasAcl=false; isEncrypted=true; isErasureCoded=false} isEmptyDirectory=FALSE eTag=null versionId=null
        at org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist(ContractTestUtils.java:1088)
        at org.apache.hadoop.fs.contract.ContractTestUtils.assertDeleted(ContractTestUtils.java:817)
        at org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest.testListEmptyRootDirectory(AbstractContractRootDirectoryTest.java:195)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)

[ERROR] org.apache.hadoop.fs.s3a.performance.ITestDirectoryMarkerListing.testRenameEmptyDirOverMarker -- Time elapsed: 0.349 s <<< FAILURE!
org.opentest4j.AssertionFailedError: Deleted file: unexpectedly found s3a://rustybucket/job-00-fork-0006/test/testRenameEmptyDirOverMarker/base/sourceDir as  S3AFileStatus{path=s3a://rustybucket/job-00-fork-0006/test/testRenameEmptyDirOverMarker/base/sourceDir; isDirectory=true; modification_time=0; access_time=0; owner=stevel; group=stevel; permission=rwxrwxrwx; isSymlink=false; hasAcl=false; isEncrypted=true; isErasureCoded=false} isEmptyDirectory=FALSE eTag=null versionId=null
        at org.junit.jupiter.api.AssertionUtils.fail(AssertionUtils.java:38)
        at org.junit.jupiter.api.Assertions.fail(Assertions.java:138)
        at org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist(ContractTestUtils.java:1088)
        at org.apache.hadoop.fs.contract.ContractTestUtils.assertDeleted(ContractTestUtils.java:817)
        at org.apache.hadoop.fs.contract.ContractTestUtils.assertDeleted(ContractTestUtils.java:790)
        at org.apache.hadoop.fs.contract.ContractTestUtils.assertDeleted(ContractTestUtils.java:772)
        at org.apache.hadoop.fs.contract.AbstractFSContractTestBase.assertDeleted(AbstractFSContractTestBase.java:366)
        at org.apache.hadoop.fs.s3a.performance.ITestDirectoryMarkerListing.testRenameEmptyDirOverMarker(ITestDirectoryMarkerListing.java:529)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)

[ERROR] org.apache.hadoop.fs.s3a.ITestS3AEmptyDirectory.testDirectoryBecomesEmpty -- Time elapsed: 0.304 s <<< FAILURE!
org.opentest4j.AssertionFailedError: Deleted file: unexpectedly found s3a://rustybucket/job-00-fork-0002/test/testEmptyDir/dir2 as  S3AFileStatus{path=s3a://rustybucket/job-00-fork-0002/test/testEmptyDir/dir2; isDirectory=true; modification_time=0; access_time=0; owner=stevel; group=stevel; permission=rwxrwxrwx; isSymlink=false; hasAcl=false; isEncrypted=true; isErasureCoded=false} isEmptyDirectory=FALSE eTag=null versionId=null
        at org.junit.jupiter.api.AssertionUtils.fail(AssertionUtils.java:38)
        at org.junit.jupiter.api.Assertions.fail(Assertions.java:138)
        at org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist(ContractTestUtils.java:1088)
        at org.apache.hadoop.fs.contract.ContractTestUtils.assertDeleted(ContractTestUtils.java:817)
        at org.apache.hadoop.fs.contract.ContractTestUtils.assertDeleted(ContractTestUtils.java:790)
        at org.apache.hadoop.fs.contract.ContractTestUtils.assertDeleted(ContractTestUtils.java:772)
        at org.apache.hadoop.fs.contract.AbstractFSContractTestBase.assertDeleted(AbstractFSContractTestBase.java:366)
        at org.apache.hadoop.fs.s3a.ITestS3AEmptyDirectory.testDirectoryBecomesEmpty(ITestS3AEmptyDirectory.java:48)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)

[WARNING] Tests run: 4, Failures: 0, Errors: 0, Skipped: 4, Time elapsed: 0.288 s -- in org.apache.hadoop.fs.s3a.ITestS3AEncryptionWithDefaultS3Settings
[ERROR] Tests run: 4, Failures: 1, Errors: 0, Skipped: 0, Time elapsed: 4.334 s <<< FAILURE! -- in org.apache.hadoop.fs.s3a.performance.ITestS3ADeleteCost
[ERROR] org.apache.hadoop.fs.s3a.performance.ITestS3ADeleteCost.testDirMarkersSubdir -- Time elapsed: 0.572 s <<< FAILURE!
java.lang.AssertionError: Expected a java.io.FileNotFoundException to be thrown, but got the result: : "[]"
        at org.apache.hadoop.test.LambdaTestUtils.intercept(LambdaTestUtils.java:505)
        at org.apache.hadoop.test.LambdaTestUtils.intercept(LambdaTestUtils.java:390)
        at org.apache.hadoop.fs.s3a.performance.ITestS3ADeleteCost.verifyNoListing(ITestS3ADeleteCost.java:210)
        at org.apache.hadoop.fs.s3a.performance.ITestS3ADeleteCost.testDirMarkersSubdir(ITestS3ADeleteCost.java:195)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)

[ERROR] org.apache.hadoop.fs.s3a.ITestS3AFileSystemContract.testRenameDirectoryMoveToExistingDirectory -- Time elapsed: 0.389 s <<< FAILURE!
org.opentest4j.AssertionFailedError: Source exists ==> expected: <false> but was: <true>
        at org.apache.hadoop.fs.FileSystemContractBaseTest.rename(FileSystemContractBaseTest.java:593)
        at org.apache.hadoop.fs.FileSystemContractBaseTest.testRenameDirectoryMoveToExistingDirectory(FileSystemContractBaseTest.java:516)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)

[ERROR] org.apache.hadoop.fs.s3a.ITestS3AFileSystemContract.testFilesystemIsCaseSensitive -- Time elapsed: 0.140 s <<< FAILURE!
org.opentest4j.AssertionFailedError: File existss3a://rustybucket/job-00-fork-0003/test/testfilesystemiscasesensitive ==> expected: <false> but was: <true>
        at org.apache.hadoop.fs.FileSystemContractBaseTest.testFilesystemIsCaseSensitive(FileSystemContractBaseTest.java:643)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)

[ERROR] org.apache.hadoop.fs.s3a.ITestS3AFileSystemContract.testDeleteRecursively -- Time elapsed: 0.200 s <<< FAILURE!
org.opentest4j.AssertionFailedError: Subdir doesn't exist ==> expected: <false> but was: <true>
        at org.apache.hadoop.fs.FileSystemContractBaseTest.testDeleteRecursively(FileSystemContractBaseTest.java:430)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)

[INFO] Running org.apache.hadoop.fs.s3a.performance.ITestCreateSessionTimeout
[INFO] Running org.apache.hadoop.fs.s3a.performance.ITestS3AMkdirCost
[INFO] Tests run: 1, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 4.637 s -- in org.apache.hadoop.fs.s3a.ITestS3APrefetchingLruEviction
[INFO] Running org.apache.hadoop.fs.s3a.ITestS3APrefetchingLruEviction
[WARNING] Tests run: 1, Failures: 0, Errors: 0, Skipped: 1, Time elapsed: 0.784 s -- in org.apache.hadoop.fs.s3a.performance.ITestCreateSessionTimeout
[INFO] Tests run: 6, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 64.71 s -- in org.apache.hadoop.fs.s3a.scale.ITestS3ADirectoryPerformance
[INFO] Running org.apache.hadoop.fs.s3a.performance.ITestS3ARenameCost
[INFO] Running org.apache.hadoop.fs.s3a.performance.ITestS3AMiscOperationCost
[INFO] Tests run: 8, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 3.535 s -- in org.apache.hadoop.fs.s3a.performance.ITestS3AOpenCost
[ERROR] Tests run: 44, Failures: 5, Errors: 0, Skipped: 0, Time elapsed: 8.030 s <<< FAILURE! -- in org.apache.hadoop.fs.s3a.ITestS3AFSMainOperations
[ERROR] org.apache.hadoop.fs.s3a.ITestS3AFSMainOperations.testDeleteEmptyDirectory -- Time elapsed: 0.127 s <<< FAILURE!
org.opentest4j.AssertionFailedError: Dir doesn't exist ==> expected: <false> but was: <true>
        at org.apache.hadoop.fs.FSMainOperationsBaseTest.testDeleteEmptyDirectory(FSMainOperationsBaseTest.java:794)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)

[ERROR] org.apache.hadoop.fs.s3a.ITestS3AFSMainOperations.testRenameDirectoryAsEmptyDirectory -- Time elapsed: 0.346 s <<< FAILURE!
org.opentest4j.AssertionFailedError: Source exists ==> expected: <false> but was: <true>
        at org.apache.hadoop.fs.FSMainOperationsBaseTest.rename(FSMainOperationsBaseTest.java:1162)
        at org.apache.hadoop.fs.FSMainOperationsBaseTest.testRenameDirectoryAsEmptyDirectory(FSMainOperationsBaseTest.java:1039)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)

[ERROR] org.apache.hadoop.fs.s3a.ITestS3AFSMainOperations.testRenameDirectoryAsNonExistentDirectory -- Time elapsed: 0.195 s <<< FAILURE!
org.opentest4j.AssertionFailedError: Source exists ==> expected: <false> but was: <true>
        at org.apache.hadoop.fs.FSMainOperationsBaseTest.rename(FSMainOperationsBaseTest.java:1162)
        at org.apache.hadoop.fs.FSMainOperationsBaseTest.doTestRenameDirectoryAsNonExistentDirectory(FSMainOperationsBaseTest.java:1007)
        at org.apache.hadoop.fs.FSMainOperationsBaseTest.testRenameDirectoryAsNonExistentDirectory(FSMainOperationsBaseTest.java:990)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)

[ERROR] org.apache.hadoop.fs.s3a.ITestS3AFSMainOperations.testGlobStatusFilterWithMultiplePathMatchesAndNonTrivialFilter -- Time elapsed: 0.073 s <<< FAILURE!
org.opentest4j.AssertionFailedError: expected: <2> but was: <0>
        at org.apache.hadoop.fs.FSMainOperationsBaseTest.testGlobStatusFilterWithMultiplePathMatchesAndNonTrivialFilter(FSMainOperationsBaseTest.java:584)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)

[ERROR] org.apache.hadoop.fs.s3a.ITestS3AFSMainOperations.testDeleteRecursively -- Time elapsed: 0.230 s <<< FAILURE!
org.opentest4j.AssertionFailedError: Subdir doesn't exist ==> expected: <false> but was: <true>
        at org.apache.hadoop.fs.FSMainOperationsBaseTest.testDeleteRecursively(FSMainOperationsBaseTest.java:785)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)

[ERROR] Tests run: 10, Failures: 1, Errors: 0, Skipped: 0, Time elapsed: 6.556 s <<< FAILURE! -- in org.apache.hadoop.fs.contract.s3a.ITestS3AContractRename
[ERROR] org.apache.hadoop.fs.contract.s3a.ITestS3AContractRename.testRenamePopulatesDirectoryAncestors -- Time elapsed: 0.433 s <<< FAILURE!
org.opentest4j.AssertionFailedError: src path should not exist: unexpectedly found s3a://rustybucket/job-00-fork-0008/test/testRenamePopulatesDirectoryAncestors/source/dir1/dir2/dir3/dir4 as  S3AFileStatus{path=s3a://rustybucket/job-00-fork-0008/test/testRenamePopulatesDirectoryAncestors/source/dir1/dir2/dir3/dir4; isDirectory=true; modification_time=0; access_time=0; owner=stevel; group=stevel; permission=rwxrwxrwx; isSymlink=false; hasAcl=false; isEncrypted=true; isErasureCoded=false} isEmptyDirectory=FALSE eTag=null versionId=null
        at org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist(ContractTestUtils.java:1088)
        at org.apache.hadoop.fs.contract.AbstractFSContractTestBase.assertPathDoesNotExist(AbstractFSContractTestBase.java:322)
        at org.apache.hadoop.fs.contract.AbstractContractRenameTest.validateAncestorsMoved(AbstractContractRenameTest.java:290)
        at org.apache.hadoop.fs.contract.AbstractContractRenameTest.testRenamePopulatesDirectoryAncestors(AbstractContractRenameTest.java:257)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)

Conclusion: listings can see recently deleted files.

  1. list multipart upload operation may not work or is inconsistent.

ListMultipartUploads operation sometimes returns an empty list even when MPUs are in progress, sometimes returns values when expected to be empty.


[ERROR] org.apache.hadoop.fs.contract.s3a.ITestS3AContractMultipartUploader.testAbortAllPendingUploads -- Time elapsed: 0.249 s <<< FAILURE!
java.lang.AssertionError: 
[Number of uploads aborted] 
Expecting:
 <0>
to be greater than or equal to:
 <1> 
        at org.apache.hadoop.fs.contract.AbstractContractMultipartUploaderTest.testAbortAllPendingUploads(AbstractContractMultipartUploaderTest.java:670)


[ERROR] Tests run: 14, Failures: 2, Errors: 0, Skipped: 3, Time elapsed: 4.687 s <<< FAILURE! -- in org.apache.hadoop.fs.s3a.s3guard.ITestS3GuardTool
[ERROR] org.apache.hadoop.fs.s3a.s3guard.ITestS3GuardTool.testUploadListByAge -- Time elapsed: 0.160 s <<< FAILURE!
org.opentest4j.AssertionFailedError: Should be one upload ==> expected: <1> but was: <0>
        at org.junit.jupiter.api.AssertionFailureBuilder.build(AssertionFailureBuilder.java:151)
        at org.junit.jupiter.api.AssertionFailureBuilder.buildAndThrow(AssertionFailureBuilder.java:132)
        at org.junit.jupiter.api.AssertEquals.failNotEqual(AssertEquals.java:197)
        at org.junit.jupiter.api.AssertEquals.assertEquals(AssertEquals.java:150)
        at org.junit.jupiter.api.Assertions.assertEquals(Assertions.java:563)
        at org.apache.hadoop.fs.s3a.s3guard.ITestS3GuardTool.testUploadListByAge(ITestS3GuardTool.java:201)


[ERROR] org.apache.hadoop.fs.s3a.s3guard.ITestS3GuardTool.testUploads -- Time elapsed: 0.172 s <<< FAILURE!
org.opentest4j.AssertionFailedError: Should be one upload ==> expected: <1> but was: <0>
        at org.junit.jupiter.api.AssertionFailureBuilder.build(AssertionFailureBuilder.java:151)
        at org.junit.jupiter.api.AssertionFailureBuilder.buildAndThrow(AssertionFailureBuilder.java:132)
        at org.junit.jupiter.api.AssertEquals.failNotEqual(AssertEquals.java:197)
        at org.junit.jupiter.api.AssertEquals.assertEquals(AssertEquals.java:150)
        at org.junit.jupiter.api.Assertions.assertEquals(Assertions.java:563)
        at org.apache.hadoop.fs.s3a.s3guard.ITestS3GuardTool.testUploads(ITestS3GuardTool.java:155)

[ERROR] org.apache.hadoop.fs.s3a.commit.staging.integration.ITestPartitionedCommitProtocol.testCommitLifecycle -- Time elapsed: 1.057 s <<< FAILURE!
org.opentest4j.AssertionFailedError: No multipart uploads in progress under s3a://rustybucket/job-00-fork-0002/test/ITestPartitionedCommitProtocol-testCommitLifecycle ==> expected: <true> but was: <false>
        at org.apache.hadoop.fs.s3a.commit.AbstractCommitITest.assertMultipartUploadsPending(AbstractCommitITest.java:238)
        at org.apache.hadoop.fs.s3a.commit.AbstractITCommitProtocol.testCommitLifecycle(AbstractITCommitProtocol.java:849)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)

[ERROR] org.apache.hadoop.fs.s3a.commit.staging.integration.ITestPartitionedCommitProtocol.testParallelJobsToAdjacentPaths -- Time elapsed: 1.191 s <<< FAILURE!
org.opentest4j.AssertionFailedError: No multipart uploads in progress under s3a://rustybucket/job-00-fork-0002/test/ITestPartitionedCommitProtocol-testParallelJobsToAdjacentPaths ==> expected: <true> but was: <false>
        at org.apache.hadoop.fs.s3a.commit.AbstractCommitITest.assertMultipartUploadsPending(AbstractCommitITest.java:238)
        at org.apache.hadoop.fs.s3a.commit.AbstractITCommitProtocol.testParallelJobsToAdjacentPaths(AbstractITCommitProtocol.java:1554)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)

[ERROR] Tests run: 24, Failures: 2, Errors: 0, Skipped: 1, Time elapsed: 22.62 s <<< FAILURE! -- in org.apache.hadoop.fs.s3a.commit.staging.integration.ITestStagingCommitProtocol
[ERROR] org.apache.hadoop.fs.s3a.commit.staging.integration.ITestStagingCommitProtocol.testCommitLifecycle -- Time elapsed: 0.832 s <<< FAILURE!
org.opentest4j.AssertionFailedError: No multipart uploads in progress under s3a://rustybucket/job-00-fork-0003/test/ITestStagingCommitProtocol-testCommitLifecycle ==> expected: <true> but was: <false>
        at org.apache.hadoop.fs.s3a.commit.AbstractCommitITest.assertMultipartUploadsPending(AbstractCommitITest.java:238)
        at org.apache.hadoop.fs.s3a.commit.AbstractITCommitProtocol.testCommitLifecycle(AbstractITCommitProtocol.java:849)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)

[ERROR] org.apache.hadoop.fs.s3a.commit.staging.integration.ITestStagingCommitProtocol.testParallelJobsToAdjacentPaths -- Time elapsed: 1.448 s <<< FAILURE!
org.opentest4j.AssertionFailedError: No multipart uploads in progress under s3a://rustybucket/job-00-fork-0003/test/ITestStagingCommitProtocol-testParallelJobsToAdjacentPaths ==> expected: <true> but was: <false>
        at org.apache.hadoop.fs.s3a.commit.AbstractCommitITest.assertMultipartUploadsPending(AbstractCommitITest.java:238)
        at org.apache.hadoop.fs.s3a.commit.AbstractITCommitProtocol.testParallelJobsToAdjacentPaths(AbstractITCommitProtocol.java:1554)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)

[WARNING] Tests run: 3, Failures: 0, Errors: 0, Skipped: 1, Time elapsed: 0.703 s -- in org.apache.hadoop.fs.s3a.commit.ITestUploadRecovery
[INFO] Running org.apache.hadoop.fs.s3a.commit.ITestUploadRecovery
[INFO] Running org.apache.hadoop.fs.s3a.impl.ITestUploadPurgeOnDirectoryOperations
[WARNING] Tests run: 3, Failures: 0, Errors: 0, Skipped: 1, Time elapsed: 0.723 s -- in org.apache.hadoop.fs.s3a.commit.ITestUploadRecovery
[INFO] Tests run: 0, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 4.923 s -- in org.apache.hadoop.fs.s3a.commit.ITestUploadRecovery
[INFO] Running org.apache.hadoop.fs.s3a.impl.ITestS3APutIfMatchAndIfNoneMatch
[ERROR] Tests run: 25, Failures: 2, Errors: 0, Skipped: 1, Time elapsed: 23.56 s <<< FAILURE! -- in org.apache.hadoop.fs.s3a.commit.staging.integration.ITestDirectoryCommitProtocol
[ERROR] org.apache.hadoop.fs.s3a.commit.staging.integration.ITestDirectoryCommitProtocol.testCommitLifecycle -- Time elapsed: 0.796 s <<< FAILURE!
org.opentest4j.AssertionFailedError: No multipart uploads in progress under s3a://rustybucket/job-00-fork-0005/test/ITestDirectoryCommitProtocol-testCommitLifecycle ==> expected: <true> but was: <false>
        at org.apache.hadoop.fs.s3a.commit.AbstractCommitITest.assertMultipartUploadsPending(AbstractCommitITest.java:238)
        at org.apache.hadoop.fs.s3a.commit.AbstractITCommitProtocol.testCommitLifecycle(AbstractITCommitProtocol.java:849)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)

[ERROR] org.apache.hadoop.fs.s3a.commit.staging.integration.ITestDirectoryCommitProtocol.testParallelJobsToAdjacentPaths -- Time elapsed: 1.421 s <<< FAILURE!
org.opentest4j.AssertionFailedError: No multipart uploads in progress under s3a://rustybucket/job-00-fork-0005/test/ITestDirectoryCommitProtocol-testParallelJobsToAdjacentPaths ==> expected: <true> but was: <false>
        at org.apache.hadoop.fs.s3a.commit.AbstractCommitITest.assertMultipartUploadsPending(AbstractCommitITest.java:238)
        at org.apache.hadoop.fs.s3a.commit.AbstractITCommitProtocol.testParallelJobsToAdjacentPaths(AbstractITCommitProtocol.java:1554)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)

[INFO] Running org.apache.hadoop.fs.s3a.impl.ITestTreewalkProblems
[INFO] Running org.apache.hadoop.fs.s3a.impl.ITestStoreClose
[WARNING] Tests run: 3, Failures: 0, Errors: 0, Skipped: 2, Time elapsed: 3.562 s -- in org.apache.hadoop.fs.s3a.impl.ITestS3AConditionalCreateBehavior
[INFO] Running org.apache.hadoop.fs.s3a.impl.ITestS3AConditionalCreateBehavior
[ERROR] Tests run: 2, Failures: 2, Errors: 0, Skipped: 0, Time elapsed: 4.074 s <<< FAILURE! -- in org.apache.hadoop.fs.s3a.impl.ITestUploadPurgeOnDirectoryOperations
[ERROR] org.apache.hadoop.fs.s3a.impl.ITestUploadPurgeOnDirectoryOperations.testRenameWithPendingUpload -- Time elapsed: 3.464 s <<< FAILURE!
java.lang.AssertionError: 
[uploads under s3a://rustybucket/job-00-fork-0002/test/testRenameWithPendingUpload/src] 
Expected size:<1> but was:<0> in:
<[]>
        at org.apache.hadoop.fs.s3a.impl.ITestUploadPurgeOnDirectoryOperations.assertUploadCount(ITestUploadPurgeOnDirectoryOperations.java:133)
        at org.apache.hadoop.fs.s3a.impl.ITestUploadPurgeOnDirectoryOperations.testRenameWithPendingUpload(ITestUploadPurgeOnDirectoryOperations.java:111)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)

[ERROR] org.apache.hadoop.fs.s3a.impl.ITestUploadPurgeOnDirectoryOperations.testDeleteWithPendingUpload -- Time elapsed: 0.488 s <<< FAILURE!
java.lang.AssertionError: 
[uploads under s3a://rustybucket/job-00-fork-0002/test/testDeleteWithPendingUpload] 
Expected size:<1> but was:<0> in:
<[]>
        at org.apache.hadoop.fs.s3a.impl.ITestUploadPurgeOnDirectoryOperations.assertUploadCount(ITestUploadPurgeOnDirectoryOperations.java:133)
        at org.apache.hadoop.fs.s3a.impl.ITestUploadPurgeOnDirectoryOperations.testDeleteWithPendingUpload(ITestUploadPurgeOnDirectoryOperations.java:86)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)

[INFO] Tests run: 3, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 1.868 s -- in org.apache.hadoop.fs.s3a.impl.ITestS3AConditionalCreateBehavior
[INFO] Tests run: 0, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 5.564 s -- in org.apache.hadoop.fs.s3a.impl.ITestS3AConditionalCreateBehavior
[INFO] Running org.apache.hadoop.fs.s3a.impl.ITestConnectionTimeouts
[INFO] Running org.apache.hadoop.fs.s3a.impl.ITestPartialRenamesDeletes
[INFO] Running org.apache.hadoop.fs.s3a.impl.ITestPartialRenamesDeletes
[WARNING] Tests run: 15, Failures: 0, Errors: 0, Skipped: 15, Time elapsed: 6.325 s -- in org.apache.hadoop.fs.s3a.impl.ITestS3APutIfMatchAndIfNoneMatch
[INFO] Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 5.289 s -- in org.apache.hadoop.fs.s3a.impl.ITestStoreClose
[ERROR] Tests run: 24, Failures: 6, Errors: 0, Skipped: 1, Time elapsed: 27.16 s <<< FAILURE! -- in org.apache.hadoop.fs.s3a.commit.magic.ITestMagicCommitProtocol
[ERROR] org.apache.hadoop.fs.s3a.commit.magic.ITestMagicCommitProtocol.testCommitLifecycle -- Time elapsed: 0.845 s <<< FAILURE!
org.opentest4j.AssertionFailedError: No multipart uploads in progress under s3a://rustybucket/job-00-fork-0008/test/ITestMagicCommitProtocol-testCommitLifecycle ==> expected: <true> but was: <false>
        at org.apache.hadoop.fs.s3a.commit.AbstractCommitITest.assertMultipartUploadsPending(AbstractCommitITest.java:238)
        at org.apache.hadoop.fs.s3a.commit.AbstractITCommitProtocol.testCommitLifecycle(AbstractITCommitProtocol.java:849)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at java.base/java.util.Optional.ifPresent(Optional.java:178)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.Spliterators$ArraySpliterator.forEachRemaining(Spliterators.java:992)
        at java.base/java.util.stream.ReferencePipeline$Head.forEach(ReferencePipeline.java:762)
        at java.base/java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:276)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.Spliterators$ArraySpliterator.forEachRemaining(Spliterators.java:992)
        at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509)
        at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499)
        at java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
        at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
        at java.base/java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:596)
        at java.base/java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:276)
        at java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1625)
        at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509)
        at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499)
        at java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
        at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
        at java.base/java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:596)
        at java.base/java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:276)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1625)
        at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509)
        at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499)
        at java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
        at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
        at java.base/java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:596)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)

[ERROR] org.apache.hadoop.fs.s3a.commit.magic.ITestMagicCommitProtocol.testAbortTaskThenJob -- Time elapsed: 0.830 s <<< FAILURE!
org.opentest4j.AssertionFailedError: magic dir : unexpectedly found s3a://rustybucket/job-00-fork-0008/test/ITestMagicCommitProtocol-testAbortTaskThenJob/__magic_job-job_202512290159_0008 as  S3AFileStatus{path=s3a://rustybucket/job-00-fork-0008/test/ITestMagicCommitProtocol-testAbortTaskThenJob/__magic_job-job_202512290159_0008; isDirectory=true; modification_time=0; access_time=0; owner=stevel; group=stevel; permission=rwxrwxrwx; isSymlink=false; hasAcl=false; isEncrypted=true; isErasureCoded=false} isEmptyDirectory=FALSE eTag=null versionId=null
        at org.junit.jupiter.api.AssertionUtils.fail(AssertionUtils.java:38)
        at org.junit.jupiter.api.Assertions.fail(Assertions.java:138)
        at org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist(ContractTestUtils.java:1088)
        at org.apache.hadoop.fs.s3a.commit.magic.ITestMagicCommitProtocol.assertJobAbortCleanedUp(ITestMagicCommitProtocol.java:112)
        at org.apache.hadoop.fs.s3a.commit.AbstractITCommitProtocol.testAbortTaskThenJob(AbstractITCommitProtocol.java:1223)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at java.base/java.util.Optional.ifPresent(Optional.java:178)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.Spliterators$ArraySpliterator.forEachRemaining(Spliterators.java:992)
        at java.base/java.util.stream.ReferencePipeline$Head.forEach(ReferencePipeline.java:762)
        at java.base/java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:276)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.Spliterators$ArraySpliterator.forEachRemaining(Spliterators.java:992)
        at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509)
        at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499)
        at java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
        at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
        at java.base/java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:596)
        at java.base/java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:276)
        at java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1625)
        at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509)
        at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499)
        at java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
        at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
        at java.base/java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:596)
        at java.base/java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:276)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1625)
        at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509)
        at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499)
        at java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
        at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
        at java.base/java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:596)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)

[ERROR] org.apache.hadoop.fs.s3a.commit.magic.ITestMagicCommitProtocol.testParallelJobsToAdjacentPaths -- Time elapsed: 1.270 s <<< FAILURE!
org.opentest4j.AssertionFailedError: No multipart uploads in progress under s3a://rustybucket/job-00-fork-0008/test/ITestMagicCommitProtocol-testParallelJobsToAdjacentPaths ==> expected: <true> but was: <false>
        at org.apache.hadoop.fs.s3a.commit.AbstractCommitITest.assertMultipartUploadsPending(AbstractCommitITest.java:238)
        at org.apache.hadoop.fs.s3a.commit.AbstractITCommitProtocol.testParallelJobsToAdjacentPaths(AbstractITCommitProtocol.java:1554)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at java.base/java.util.Optional.ifPresent(Optional.java:178)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.Spliterators$ArraySpliterator.forEachRemaining(Spliterators.java:992)
        at java.base/java.util.stream.ReferencePipeline$Head.forEach(ReferencePipeline.java:762)
        at java.base/java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:276)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.Spliterators$ArraySpliterator.forEachRemaining(Spliterators.java:992)
        at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509)
        at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499)
        at java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
        at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
        at java.base/java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:596)
        at java.base/java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:276)
        at java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1625)
        at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509)
        at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499)
        at java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
        at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
        at java.base/java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:596)
        at java.base/java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:276)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1625)
        at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509)
        at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499)
        at java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
        at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
        at java.base/java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:596)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)

[ERROR] org.apache.hadoop.fs.s3a.commit.magic.ITestMagicCommitProtocol.testRecoveryAndCleanup -- Time elapsed: 0.811 s <<< FAILURE!
org.opentest4j.AssertionFailedError: task attempt dir: unexpectedly found s3a://rustybucket/job-00-fork-0008/test/ITestMagicCommitProtocol-testRecoveryAndCleanup/__magic_job-job_202512290106_0008/job-job_202512290106_0008/01/tasks/attempt_202512290106_0008_m_000000_0/__base as  S3AFileStatus{path=s3a://rustybucket/job-00-fork-0008/test/ITestMagicCommitProtocol-testRecoveryAndCleanup/__magic_job-job_202512290106_0008/job-job_202512290106_0008/01/tasks/attempt_202512290106_0008_m_000000_0/__base; isDirectory=true; modification_time=0; access_time=0; owner=stevel; group=stevel; permission=rwxrwxrwx; isSymlink=false; hasAcl=false; isEncrypted=true; isErasureCoded=false} isEmptyDirectory=FALSE eTag=null versionId=null
        at org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist(ContractTestUtils.java:1088)
        at org.apache.hadoop.fs.s3a.commit.AbstractITCommitProtocol.assertTaskAttemptPathDoesNotExist(AbstractITCommitProtocol.java:695)
        at org.apache.hadoop.fs.s3a.commit.AbstractITCommitProtocol.testRecoveryAndCleanup(AbstractITCommitProtocol.java:655)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at java.base/java.util.Optional.ifPresent(Optional.java:178)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.Spliterators$ArraySpliterator.forEachRemaining(Spliterators.java:992)
        at java.base/java.util.stream.ReferencePipeline$Head.forEach(ReferencePipeline.java:762)
        at java.base/java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:276)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.Spliterators$ArraySpliterator.forEachRemaining(Spliterators.java:992)
        at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509)
        at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499)
        at java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
        at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
        at java.base/java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:596)
        at java.base/java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:276)
        at java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1625)
        at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509)
        at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499)
        at java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
        at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
        at java.base/java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:596)
        at java.base/java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:276)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1625)
        at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509)
        at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499)
        at java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
        at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
        at java.base/java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:596)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)

[ERROR] org.apache.hadoop.fs.s3a.commit.magic.ITestMagicCommitProtocol.testAbortJobNotTask -- Time elapsed: 0.612 s <<< FAILURE!
org.opentest4j.AssertionFailedError: task attempt dir: unexpectedly found s3a://rustybucket/job-00-fork-0008/test/ITestMagicCommitProtocol-testAbortJobNotTask/__magic_job-job_202512290879_0008/job-job_202512290879_0008/01/tasks/attempt_202512290879_0008_m_000000_0/__base as  S3AFileStatus{path=s3a://rustybucket/job-00-fork-0008/test/ITestMagicCommitProtocol-testAbortJobNotTask/__magic_job-job_202512290879_0008/job-job_202512290879_0008/01/tasks/attempt_202512290879_0008_m_000000_0/__base; isDirectory=true; modification_time=0; access_time=0; owner=stevel; group=stevel; permission=rwxrwxrwx; isSymlink=false; hasAcl=false; isEncrypted=true; isErasureCoded=false} isEmptyDirectory=FALSE eTag=null versionId=null
        at org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist(ContractTestUtils.java:1088)
        at org.apache.hadoop.fs.s3a.commit.AbstractITCommitProtocol.assertTaskAttemptPathDoesNotExist(AbstractITCommitProtocol.java:695)
        at org.apache.hadoop.fs.s3a.commit.AbstractITCommitProtocol.lambda$testAbortJobNotTask$11(AbstractITCommitProtocol.java:1294)
        at org.apache.hadoop.fs.s3a.commit.AbstractITCommitProtocol.executeWork(AbstractITCommitProtocol.java:624)
        at org.apache.hadoop.fs.s3a.commit.AbstractITCommitProtocol.executeWork(AbstractITCommitProtocol.java:608)
        at org.apache.hadoop.fs.s3a.commit.AbstractITCommitProtocol.testAbortJobNotTask(AbstractITCommitProtocol.java:1289)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at java.base/java.util.Optional.ifPresent(Optional.java:178)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.Spliterators$ArraySpliterator.forEachRemaining(Spliterators.java:992)
        at java.base/java.util.stream.ReferencePipeline$Head.forEach(ReferencePipeline.java:762)
        at java.base/java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:276)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.Spliterators$ArraySpliterator.forEachRemaining(Spliterators.java:992)
        at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509)
        at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499)
        at java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
        at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
        at java.base/java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:596)
        at java.base/java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:276)
        at java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1625)
        at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509)
        at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499)
        at java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
        at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
        at java.base/java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:596)
        at java.base/java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:276)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1625)
        at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509)
        at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499)
        at java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
        at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
        at java.base/java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:596)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)

[ERROR] org.apache.hadoop.fs.s3a.commit.magic.ITestMagicCommitProtocol.testCommitterWithNoOutputs -- Time elapsed: 0.752 s <<< FAILURE!
org.opentest4j.AssertionFailedError: task attempt dir: unexpectedly found s3a://rustybucket/job-00-fork-0008/test/ITestMagicCommitProtocol-testCommitterWithNoOutputs/__magic_job-job_202512290330_0008/job-job_202512290330_0008/01/tasks/attempt_202512290330_0008_m_000000_0/__base as  S3AFileStatus{path=s3a://rustybucket/job-00-fork-0008/test/ITestMagicCommitProtocol-testCommitterWithNoOutputs/__magic_job-job_202512290330_0008/job-job_202512290330_0008/01/tasks/attempt_202512290330_0008_m_000000_0/__base; isDirectory=true; modification_time=0; access_time=0; owner=stevel; group=stevel; permission=rwxrwxrwx; isSymlink=false; hasAcl=false; isEncrypted=true; isErasureCoded=false} isEmptyDirectory=FALSE eTag=null versionId=null
        at org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist(ContractTestUtils.java:1088)
        at org.apache.hadoop.fs.s3a.commit.AbstractITCommitProtocol.assertTaskAttemptPathDoesNotExist(AbstractITCommitProtocol.java:695)
        at org.apache.hadoop.fs.s3a.commit.AbstractITCommitProtocol.testCommitterWithNoOutputs(AbstractITCommitProtocol.java:1080)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at java.base/java.util.Optional.ifPresent(Optional.java:178)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.Spliterators$ArraySpliterator.forEachRemaining(Spliterators.java:992)
        at java.base/java.util.stream.ReferencePipeline$Head.forEach(ReferencePipeline.java:762)
        at java.base/java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:276)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.Spliterators$ArraySpliterator.forEachRemaining(Spliterators.java:992)
        at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509)
        at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499)
        at java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
        at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
        at java.base/java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:596)
        at java.base/java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:276)
        at java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1625)
        at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509)
        at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499)
        at java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
        at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
        at java.base/java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:596)
        at java.base/java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:276)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1625)
        at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509)
        at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499)
        at java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
        at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
        at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
        at java.base/java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:596)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)

all of these are failures saying


[ERROR] org.apache.hadoop.fs.s3a.impl.ITestS3AStoreWriterService.testListMultipartUploads -- Time elapsed: 4.347 s <<< FAILURE!
java.lang.AssertionError: 
[Uploads which we expected to be listed.] 
Expecting empty but was:<[IdKey{key='job-00-fork-0006/test/pending-part-0', uploadId='Mzk1NmVjMDYtYzAxOS00N2NjLWI0OGQtM2ExNTcxN2ExN2QyLjNlYmI5ZWJhLWRlZWEtNDVhYi05MDJlLTA1NjU0MWVjMmUzMXgxNzY3MDMyOTA4MjMyOTEzMDAw'},
    IdKey{key='job-00-fork-0006/test/pending-part-1', uploadId='Mzk1NmVjMDYtYzAxOS00N2NjLWI0OGQtM2ExNTcxN2ExN2QyLjk4ODA3Nzk2LWY2NGMtNDQ0NC04OWEzLTNjNWQ1YTEwZmI3NHgxNzY3MDMyOTA4Mjg2NDMxMDAw'},
    IdKey{key='job-00-fork-0006/test/pending-part-2', uploadId='Mzk1NmVjMDYtYzAxOS00N2NjLWI0OGQtM2ExNTcxN2ExN2QyLjI3ZGYxODMwLWM3YjEtNGYzZC05MjAzLTdiNzI5MjdmZGNmMXgxNzY3MDMyOTA4MzA2OTQwMDAw'},
    IdKey{key='job-00-fork-0006/test/pending-part-4', uploadId='Mzk1NmVjMDYtYzAxOS00N2NjLWI0OGQtM2ExNTcxN2ExN2QyLjQ5MDQ3Njg0LWU5NTAtNGY0NC1hZDM1LTU4ZDk2ZjUwOTZjM3gxNzY3MDMyOTA4NTE5MjE5MDAw'},
    IdKey{key='job-00-fork-0006/test/pending-part-3', uploadId='Mzk1NmVjMDYtYzAxOS00N2NjLWI0OGQtM2ExNTcxN2ExN2QyLmJkZGU5Mjg1LTQ1MDUtNGM4NC04OTVkLWU1YWM3MTUxMGVjNngxNzY3MDMyOTA4NDQ4ODAyMDAw'}]>
        at org.apache.hadoop.fs.s3a.impl.ITestS3AStoreWriterService.assertUploadsPresent(ITestS3AStoreWriterService.java:131)
        at org.apache.hadoop.fs.s3a.impl.ITestS3AStoreWriterService.testListMultipartUploads(ITestS3AStoreWriterService.java:95)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)


  1. filesystem is case insensitive, at least on a mac
[ERROR]   ITestS3AFileSystemContract>FileSystemContractBaseTest.testFilesystemIsCaseSensitive:643 File exists  s3a://rustybucket/job-00-fork-0003/test/testfilesystemiscasesensitive ==> expected: <false> but was: <true> 
  1. getBucketMetadata operation unsupported; returns 501.
[ERROR] org.apache.hadoop.fs.s3a.ITestS3AEndpointRegion.testWithoutRegionConfig -- Time elapsed: 0.091 s <<< ERROR!
org.apache.hadoop.fs.s3a.AWSUnsupportedFeatureException: getBucketMetadata() on rustybucket: software.amazon.awssdk.services.s3.model.S3Exception: Not Implemented (Service: S3, Status Code: 501, Request ID: null):null: Not Implemented (Service: S3, Status Code: 501, Request ID: null)
        at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:311)
        at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:124)
        at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$4(Invoker.java:376)
        at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:468)
        at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:372)
        at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:347)
        at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$getBucketMetadata$10(S3AFileSystem.java:2983)
        at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration(IOStatisticsBinding.java:546)
        at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.lambda$trackDurationOfOperation$5(IOStatisticsBinding.java:527)
        at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration(IOStatisticsBinding.java:448)
        at org.apache.hadoop.fs.s3a.S3AFileSystem.trackDurationAndSpan(S3AFileSystem.java:2806)
        at org.apache.hadoop.fs.s3a.S3AFileSystem.getBucketMetadata(S3AFileSystem.java:2982)
        at org.apache.hadoop.fs.s3a.S3AFileSystem$S3AInternalsImpl.getBucketMetadata(S3AFileSystem.java:1603)
        at org.apache.hadoop.fs.s3a.ITestS3AEndpointRegion.testWithoutRegionConfig(ITestS3AEndpointRegion.java:140)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
Caused by: software.amazon.awssdk.services.s3.model.S3Exception: Not Implemented (Service: S3, Status Code: 501, Request ID: null)
        at software.amazon.awssdk.services.s3.model.S3Exception$BuilderImpl.build(S3Exception.java:113)
        at software.amazon.awssdk.services.s3.model.S3Exception$BuilderImpl.build(S3Exception.java:61)
        at software.amazon.awssdk.services.s3.internal.handlers.ExceptionTranslationInterceptor.modifyException(ExceptionTranslationInterceptor.java:88)
        at software.amazon.awssdk.core.interceptor.ExecutionInterceptorChain.modifyException(ExecutionInterceptorChain.java:181)
        at software.amazon.awssdk.core.internal.http.pipeline.stages.utils.ExceptionReportingUtils.runModifyException(ExceptionReportingUtils.java:54)
        at software.amazon.awssdk.core.internal.http.pipeline.stages.utils.ExceptionReportingUtils.reportFailureToInterceptors(ExceptionReportingUtils.java:38)
        at software.amazon.awssdk.core.internal.http.pipeline.stages.ExecutionFailureExceptionReportingStage.execute(ExecutionFailureExceptionReportingStage.java:39)
        at software.amazon.awssdk.core.internal.http.pipeline.stages.ExecutionFailureExceptionReportingStage.execute(ExecutionFailureExceptionReportingStage.java:26)
        at software.amazon.awssdk.core.internal.http.AmazonSyncHttpClient$RequestExecutionBuilderImpl.execute(AmazonSyncHttpClient.java:210)
        at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.invoke(BaseSyncClientHandler.java:103)
        at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.doExecute(BaseSyncClientHandler.java:173)
        at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.lambda$execute$1(BaseSyncClientHandler.java:80)
        at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.measureApiCallSuccess(BaseSyncClientHandler.java:182)
        at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.execute(BaseSyncClientHandler.java:74)
        at software.amazon.awssdk.core.client.handler.SdkSyncClientHandler.execute(SdkSyncClientHandler.java:45)
        at software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler.execute(AwsSyncClientHandler.java:53)
        at software.amazon.awssdk.services.s3.DefaultS3Client.headBucket(DefaultS3Client.java:7393)
        at software.amazon.awssdk.services.s3.DelegatingS3Client.lambda$headBucket$58(DelegatingS3Client.java:5969)
        at software.amazon.awssdk.services.s3.internal.crossregion.S3CrossRegionSyncClient.invokeOperation(S3CrossRegionSyncClient.java:67)
        at software.amazon.awssdk.services.s3.DelegatingS3Client.headBucket(DelegatingS3Client.java:5969)
        at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$getBucketMetadata$9(S3AFileSystem.java:2985)
        at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:122)
        ... 15 more

trivial and not unusual with third party stores.

Overall, bit of a mess at least as far as testing goes.

I would avoid using anywhere you require case sensitivitiy or list consistency, and would want to spend more time exploring the multipart test failures to be confident you can upload objects in sizes of many GB. That listing inconsistency brings back bad memories of the old S3, when hive and spark queries could miss data.

I would absolutely avoid using any of the s3a committers against it without understanding what is up with MPU listings.

@steveloughran
Copy link
Contributor Author

@ahmarsuhail assuming you have time this week, can you take a quick look?

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 1m 15s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 1s No case conflicting files found.
+0 🆗 codespell 0m 0s codespell was not available.
+0 🆗 detsecrets 0m 0s detect-secrets was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 test4tests 0m 0s The patch appears to include 17 new or modified test files.
_ trunk Compile Tests _
+0 🆗 mvndep 8m 13s Maven dependency ordering for branch
-1 ❌ mvninstall 35m 50s /branch-mvninstall-root.txt root in trunk failed.
+1 💚 compile 19m 3s trunk passed with JDK Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04
+1 💚 compile 19m 16s trunk passed with JDK Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
+1 💚 checkstyle 3m 49s trunk passed
+1 💚 mvnsite 3m 14s trunk passed
+1 💚 javadoc 2m 25s trunk passed with JDK Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04
+1 💚 javadoc 2m 18s trunk passed with JDK Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
-1 ❌ spotbugs 1m 44s /branch-spotbugs-hadoop-tools_hadoop-aws-warnings.html hadoop-tools/hadoop-aws in trunk has 2 extant spotbugs warnings.
+1 💚 shadedclient 33m 27s branch has no errors when building and testing our client artifacts.
_ Patch Compile Tests _
+0 🆗 mvndep 0m 31s Maven dependency ordering for patch
+1 💚 mvninstall 1m 54s the patch passed
+1 💚 compile 17m 57s the patch passed with JDK Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04
+1 💚 javac 17m 57s the patch passed
+1 💚 compile 19m 19s the patch passed with JDK Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
+1 💚 javac 19m 19s the patch passed
+1 💚 blanks 0m 0s The patch has no blanks issues.
-0 ⚠️ checkstyle 3m 38s /results-checkstyle-root.txt root: The patch generated 1 new + 13 unchanged - 5 fixed = 14 total (was 18)
+1 💚 mvnsite 3m 12s the patch passed
-1 ❌ javadoc 0m 55s /results-javadoc-javadoc-hadoop-tools_hadoop-aws-jdkUbuntu-21.0.7+6-Ubuntu-0ubuntu120.04.txt hadoop-tools_hadoop-aws-jdkUbuntu-21.0.7+6-Ubuntu-0ubuntu120.04 with JDK Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04 generated 2 new + 929 unchanged - 0 fixed = 931 total (was 929)
-1 ❌ javadoc 0m 58s /results-javadoc-javadoc-hadoop-tools_hadoop-aws-jdkUbuntu-17.0.15+6-Ubuntu-0ubuntu120.04.txt hadoop-tools_hadoop-aws-jdkUbuntu-17.0.15+6-Ubuntu-0ubuntu120.04 with JDK Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04 generated 2 new + 851 unchanged - 0 fixed = 853 total (was 851)
+1 💚 spotbugs 5m 16s the patch passed
+1 💚 shadedclient 32m 27s patch has no errors when building and testing our client artifacts.
_ Other Tests _
+1 💚 unit 22m 58s hadoop-common in the patch passed.
+1 💚 unit 4m 53s hadoop-aws in the patch passed.
+1 💚 asflicense 1m 14s The patch does not generate ASF License warnings.
257m 38s
Subsystem Report/Notes
Docker ClientAPI=1.52 ServerAPI=1.52 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7700/15/artifact/out/Dockerfile
GITHUB PR #7700
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets
uname Linux 1da648bef76a 5.15.0-164-generic #174-Ubuntu SMP Fri Nov 14 20:25:16 UTC 2025 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / 2290412
Default Java Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
Multi-JDK versions /usr/lib/jvm/java-21-openjdk-amd64:Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04 /usr/lib/jvm/java-17-openjdk-amd64:Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7700/15/testReport/
Max. process+thread count 3144 (vs. ulimit of 5500)
modules C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-aws U: .
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7700/15/console
versions git=2.25.1 maven=3.9.11 spotbugs=4.9.7
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

@apache apache deleted a comment from hadoop-yetus Dec 31, 2025
@apache apache deleted a comment from hadoop-yetus Dec 31, 2025
@apache apache deleted a comment from hadoop-yetus Dec 31, 2025
@apache apache deleted a comment from hadoop-yetus Dec 31, 2025
@apache apache deleted a comment from hadoop-yetus Dec 31, 2025
@apache apache deleted a comment from hadoop-yetus Dec 31, 2025
@apache apache deleted a comment from hadoop-yetus Dec 31, 2025
@apache apache deleted a comment from hadoop-yetus Dec 31, 2025
@apache apache deleted a comment from hadoop-yetus Dec 31, 2025
@apache apache deleted a comment from hadoop-yetus Dec 31, 2025
@apache apache deleted a comment from hadoop-yetus Dec 31, 2025
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants