Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

GH-38364: [Python] Initialize S3 on first use #38375

Merged
merged 4 commits into from
Oct 24, 2023

Conversation

pentschev
Copy link
Contributor

@pentschev pentschev commented Oct 20, 2023

Rationale for this change

In accordance to #38364, we believe that for various reasons (shortening import time, preventing unnecessary resource consumption and potential bugs with S3 library) it is appropriate to avoid initialization of S3 resources at import time and move that step to occur at first-use.

What changes are included in this PR?

  • Remove calls to ensure_s3_initialized() that were up until now executed during import pyarrow.fs;
  • Move ensure_s3_intialized() calls to python/pyarrow/_s3fs.pyx module;
  • Add global flag to mark whether S3 has been previously initialized and atexit handlers registered.

Are these changes tested?

Yes, existing S3 tests check whether it has been initialized, otherwise failing with a C++ exception.

Are there any user-facing changes?

No, the behavior is now slightly different with S3 initialization not happening immediately after pyarrow.fs is imported, but no changes are expected from a user perspective relying on the public API alone.

This PR contains a "Critical Fix".
A bug in aws-sdk-cpp reported in aws/aws-sdk-cpp#2681 causes segmentation faults under specific circumstances when Python processes shutdown, specifically observed with Dask+GPUs (so far we were unable to pinpoint the exact correlation of Dask+GPUs+S3). While this definitely doesn't seem to affect all users and is not directly sourced in Arrow, it may affect use cases that are completely independent of S3 to operate, which is particularly problematic in CI where all tests pass successfully but the process crashes at shutdown.

@github-actions
Copy link

⚠️ GitHub issue #38364 has been automatically assigned in GitHub to PR creator.

@pentschev
Copy link
Contributor Author

pentschev commented Oct 20, 2023

The failed test is a C++ test:

[ RUN      ] TestS3FS.GetFileInfoGeneratorStress
API: DeleteMultipleObjects(bucket=stress, multiObject=true, numberOfObjects=1000)
Time: 12:20:20 UTC 10/20/2023
DeploymentID: 8b5b1ae6-66cf-4ead-a4fd-3ef2b430d16c
RequestID: 178FD01F921FEEC8
RemoteHost: 127.0.0.1
Host: 127.0.0.8:51944
UserAgent: aws-sdk-cpp/1.11.68 Windows/10.0.17763.4252 AMD64 MSVC/1929
Error: remove C:\Users\appveyor\AppData\Local\Temp\1\s3fs-test-30c6atjk/.minio.sys/buckets/stress/1/230/fs.json: The process cannot access the file because it is being used by another process. (*fs.PathError)
       6: internal\logger\logger.go:278:logger.LogIf()
       5: cmd\fs-v1-helpers.go:430:cmd.fsDeleteFile()
       4: cmd\fs-v1.go:1210:cmd.(*FSObjects).DeleteObject()
       3: cmd\fs-v1.go:1142:cmd.(*FSObjects).DeleteObjects()
       2: cmd\bucket-handlers.go:592:cmd.objectAPIHandlers.DeleteMultipleObjectsHandler()
       1: net\http\server.go:2047:http.HandlerFunc.ServeHTTP()
API: DeleteMultipleObjects(bucket=stress, multiObject=true, numberOfObjects=1000)
Time: 12:20:21 UTC 10/20/2023
DeploymentID: 8b5b1ae6-66cf-4ead-a4fd-3ef2b430d16c
RequestID: 178FD01F921FEEC8
RemoteHost: 127.0.0.1
Host: 127.0.0.8:51944
UserAgent: aws-sdk-cpp/1.11.68 Windows/10.0.17763.4252 AMD64 MSVC/1929
Error: remove C:\Users\appveyor\AppData\Local\Temp\1\s3fs-test-30c6atjk/.minio.sys/buckets/stress/1/230/fs.json: The process cannot access the file because it is being used by another process. (*fs.PathError)
       5: internal\logger\logger.go:278:logger.LogIf()
       4: cmd\api-errors.go:2123:cmd.toAPIErrorCode()
       3: cmd\api-errors.go:2148:cmd.toAPIError()
       2: cmd\bucket-handlers.go:620:cmd.objectAPIHandlers.DeleteMultipleObjectsHandler()
       1: net\http\server.go:2047:http.HandlerFunc.ServeHTTP()
[FATAL] 2023-10-20 12:19:08.379 WinHttpSyncHttpClient [2536] Failed setting secure crypto protocols with error code: 87
[FATAL] 2023-10-20 12:19:08.950 WinHttpSyncHttpClient [2536] Failed setting secure crypto protocols with error code: 87
C:/projects/arrow/cpp/src/arrow/filesystem/s3fs_test.cc(865): error: Failed
'fs_->DeleteDirContents("stress")' failed with IOError: Got the following 1 errors when deleting objects in S3 bucket 'stress':
- key '1/230': We encountered an internal error, please try again.: cause(remove C:\Users\appveyor\AppData\Local\Temp\1\s3fs-test-30c6atjk/.minio.sys/buckets/stress/1/230/fs.json: The process cannot access the file because it is being used by another process.)
[  FAILED  ] TestS3FS.GetFileInfoGeneratorStress (72879 ms)

This PR is not touching any C++ code, is there any Python code being executed during C++ testing or is it purely a coincidence that the failing test is also an S3 test?

EDIT: This seems to have been a one time fluke.

@pentschev pentschev force-pushed the s3fs-init-on-use branch 2 times, most recently from 5c5f30a to 4307861 Compare October 20, 2023 13:17
Initializing S3 on import may cause undesired consequences for users who
do not use S3:
- Longer import times;
- Consumption of unnecessary resources, e.g., AWS event loop thread(s);
- Potential exposure to bugs in S3 package dependencies.

Therefore, a more appropriate way to handle S3 initialization seems to
be to move it to its first use.
@rjzamora
Copy link
Contributor

(cc @jorisvandenbossche) We are very interested in feedback on this one. The aws-sdk-cpp bug is causing RAPIDS a lot of pain, and so we are very motivated to avoid the import whenever possible.

Comment on lines 274 to 293
ensure_s3_initialized()

self._initialize_s3(access_key=access_key, secret_key=secret_key, session_token=session_token,
anonymous=anonymous, region=region, request_timeout=request_timeout,
connect_timeout=connect_timeout, scheme=scheme, endpoint_override=endpoint_override,
background_writes=background_writes, default_metadata=default_metadata,
role_arn=role_arn, session_name=session_name, external_id=external_id,
load_frequency=load_frequency, proxy_options=proxy_options,
allow_bucket_creation=allow_bucket_creation, allow_bucket_deletion=allow_bucket_deletion,
retry_strategy=retry_strategy)

def _initialize_s3(self, *, access_key=None, secret_key=None, session_token=None,
bint anonymous=False, region=None, request_timeout=None,
connect_timeout=None, scheme=None, endpoint_override=None,
bint background_writes=True, default_metadata=None,
role_arn=None, session_name=None, external_id=None,
load_frequency=900, proxy_options=None,
allow_bucket_creation=False, allow_bucket_deletion=False,
retry_strategy: S3RetryStrategy = AwsStandardS3RetryStrategy(max_attempts=3)):

Copy link
Member

Choose a reason for hiding this comment

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

Do we need this?
It seems that __init__() already calls ensure_s3_initialized().

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The reason I had to move the implementation to the _initialize_s3 method are the cdefs in https://github.com/apache/arrow/pull/38375/files#diff-afa3ea99a387be221ef1f7230aa309b42001aed318cdc6969e700d5eb04d07b2R294-R296, they will instantiate objects that expect S3 to be already initialized (which is only guaranteed after ensure_s3_initialized() is called) and Cython will instantiate them before any code in the function runs, including ensure_s3_initialized().

Another alternative would be to make them unique_ptrs or something that won't instantiate the objects immediately at __init__()'s entry, but I think this would make the code more complex than the solution currently proposed. If a pointer is preferred here without deferring to another method I can work on that.

Copy link
Member

Choose a reason for hiding this comment

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

The given cdef's are std::optional and std::shared_ptr, so I don't understand the concern.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Indeed they are, thanks for calling my attention to that. Now I see what happened, I initially tested this in the 12.0.1 tag where CS3Options was not a std::optional and that was causing the CS3Options default constructor to be called before ensure_s3_initialized(). I can confirm it works as expected in main without that, and fixed it now via 58097e9 .

python/pyarrow/fs.py Show resolved Hide resolved
@github-actions github-actions bot added awaiting changes Awaiting changes and removed awaiting review Awaiting review labels Oct 24, 2023
@github-actions github-actions bot added awaiting change review Awaiting change review and removed awaiting changes Awaiting changes labels Oct 24, 2023
@pitrou
Copy link
Member

pitrou commented Oct 24, 2023

@github-actions crossbow submit -g python -g wheel

@github-actions
Copy link

Revision: 58097e9

Submitted crossbow builds: ursacomputing/crossbow @ actions-06c6a57213

Task Status
test-conda-python-3.10 Github Actions
test-conda-python-3.10-cython2 Github Actions
test-conda-python-3.10-hdfs-2.9.2 Github Actions
test-conda-python-3.10-hdfs-3.2.1 Github Actions
test-conda-python-3.10-pandas-latest Github Actions
test-conda-python-3.10-pandas-nightly Github Actions
test-conda-python-3.10-spark-v3.5.0 Github Actions
test-conda-python-3.10-substrait Github Actions
test-conda-python-3.11 Github Actions
test-conda-python-3.11-dask-latest Github Actions
test-conda-python-3.11-dask-upstream_devel Github Actions
test-conda-python-3.11-hypothesis Github Actions
test-conda-python-3.11-pandas-upstream_devel Github Actions
test-conda-python-3.11-spark-master Github Actions
test-conda-python-3.12 Github Actions
test-conda-python-3.8 Github Actions
test-conda-python-3.8-pandas-1.0 Github Actions
test-conda-python-3.8-spark-v3.5.0 Github Actions
test-conda-python-3.9 Github Actions
test-conda-python-3.9-pandas-latest Github Actions
test-cuda-python Github Actions
test-debian-11-python-3 Azure
test-fedora-35-python-3 Azure
test-ubuntu-20.04-python-3 Azure
test-ubuntu-22.04-python-3 Github Actions
wheel-clean Github Actions
wheel-macos-big-sur-cp310-arm64 Github Actions
wheel-macos-big-sur-cp311-arm64 Github Actions
wheel-macos-big-sur-cp312-arm64 Github Actions
wheel-macos-big-sur-cp38-arm64 Github Actions
wheel-macos-big-sur-cp39-arm64 Github Actions
wheel-macos-mojave-cp310-amd64 Github Actions
wheel-macos-mojave-cp311-amd64 Github Actions
wheel-macos-mojave-cp312-amd64 Github Actions
wheel-macos-mojave-cp38-amd64 Github Actions
wheel-macos-mojave-cp39-amd64 Github Actions
wheel-manylinux-2-28-cp310-amd64 Github Actions
wheel-manylinux-2-28-cp310-arm64 Github Actions
wheel-manylinux-2-28-cp311-amd64 Github Actions
wheel-manylinux-2-28-cp311-arm64 Github Actions
wheel-manylinux-2-28-cp312-amd64 Github Actions
wheel-manylinux-2-28-cp312-arm64 Github Actions
wheel-manylinux-2-28-cp38-amd64 Github Actions
wheel-manylinux-2-28-cp38-arm64 Github Actions
wheel-manylinux-2-28-cp39-amd64 Github Actions
wheel-manylinux-2-28-cp39-arm64 Github Actions
wheel-manylinux-2014-cp310-amd64 Github Actions
wheel-manylinux-2014-cp310-arm64 Github Actions
wheel-manylinux-2014-cp311-amd64 Github Actions
wheel-manylinux-2014-cp311-arm64 Github Actions
wheel-manylinux-2014-cp312-amd64 Github Actions
wheel-manylinux-2014-cp312-arm64 Github Actions
wheel-manylinux-2014-cp38-amd64 Github Actions
wheel-manylinux-2014-cp38-arm64 Github Actions
wheel-manylinux-2014-cp39-amd64 Github Actions
wheel-manylinux-2014-cp39-arm64 Github Actions
wheel-windows-cp310-amd64 Github Actions
wheel-windows-cp311-amd64 Github Actions
wheel-windows-cp312-amd64 Github Actions
wheel-windows-cp38-amd64 Github Actions
wheel-windows-cp39-amd64 Github Actions

Copy link
Member

@pitrou pitrou left a comment

Choose a reason for hiding this comment

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

+1, thank you @pentschev

@pentschev
Copy link
Contributor Author

Thanks @pitrou and @kou for the reviews, truly appreciate the help here! 😄

@pitrou
Copy link
Member

pitrou commented Oct 24, 2023

@pentschev You're welcome!

@pitrou pitrou merged commit 7751444 into apache:main Oct 24, 2023
10 of 11 checks passed
@pitrou pitrou removed the awaiting change review Awaiting change review label Oct 24, 2023
@bdice
Copy link

bdice commented Oct 24, 2023

Huge thank you to @kou and @pitrou for your timely review and support! 🙇‍♂️

@vyasr
Copy link
Contributor

vyasr commented Oct 24, 2023

Thanks all! This is a big help for RAPIDS.

@jakirkham
Copy link

Thanks all! 🙏

@kou
Copy link
Member

kou commented Oct 24, 2023

@pentschev No problem! Could you remove the newly added ensure_s3_initialized() call ( https://github.com/apache/arrow/pull/38375/files#diff-afa3ea99a387be221ef1f7230aa309b42001aed318cdc6969e700d5eb04d07b2R272-R273 ) in the S3FileSystem.__init__() as a follow-up PR? We don't need multiple ensure_s3_initialized() calls.

ensure_s3_initialized()
cdef:
optional[CS3Options] options
shared_ptr[CS3FileSystem] wrapped
# Need to do this before initializing `options` as the S3Options
# constructor has a debug check against use after S3 finalization.
ensure_s3_initialized()

@pentschev
Copy link
Contributor Author

Apologies for overlooking that @kou , and good catch! Opened #38451 to address that now.

@conbench-apache-arrow
Copy link

After merging your PR, Conbench analyzed the 6 benchmarking runs that have been run so far on merge-commit 7751444.

There were no benchmark performance regressions. 🎉

The full Conbench report has more details. It also includes information about 6 possible false positives for unstable benchmarks that are known to sometimes produce them.

JerAguilon pushed a commit to JerAguilon/arrow that referenced this pull request Oct 25, 2023
### Rationale for this change

In accordance to apache#38364, we believe that for various reasons (shortening import time, preventing unnecessary resource consumption and potential bugs with S3 library) it is appropriate to avoid initialization of S3 resources at import time and move that step to occur at first-use.

### What changes are included in this PR?

- Remove calls to `ensure_s3_initialized()` that were up until now executed during `import pyarrow.fs`;
- Move `ensure_s3_intialized()` calls to `python/pyarrow/_s3fs.pyx` module;
- Add global flag to mark whether S3 has been previously initialized and `atexit` handlers registered.

### Are these changes tested?

Yes, existing S3 tests check whether it has been initialized, otherwise failing with a C++ exception.

### Are there any user-facing changes?

No, the behavior is now slightly different with S3 initialization not happening immediately after `pyarrow.fs` is imported, but no changes are expected from a user perspective relying on the public API alone.

**This PR contains a "Critical Fix".**
A bug in aws-sdk-cpp reported in aws/aws-sdk-cpp#2681 causes segmentation faults under specific circumstances when Python processes shutdown, specifically observed with Dask+GPUs (so far we were unable to pinpoint the exact correlation of Dask+GPUs+S3). While this definitely doesn't seem to affect all users and is not directly sourced in Arrow, it may affect use cases that are completely independent of S3 to operate, which is particularly problematic in CI where all tests pass successfully but the process crashes at shutdown.
* Closes: apache#38364

Lead-authored-by: Peter Andreas Entschev <[email protected]>
Co-authored-by: Antoine Pitrou <[email protected]>
Signed-off-by: Antoine Pitrou <[email protected]>
kou pushed a commit that referenced this pull request Oct 25, 2023
### Rationale for this change

#38375 introduced duplicate calls to `ensure_s3_initialized()`.

### What changes are included in this PR?

Deduplicates call to `ensure_s3_initialized()`.

### Are these changes tested?

Yes, covered by existing S3 tests.

### Are there any user-facing changes?

No.

Authored-by: Peter Andreas Entschev <[email protected]>
Signed-off-by: Sutou Kouhei <[email protected]>
@jorisvandenbossche
Copy link
Member

FYI we are in the process of releasing pyarrow 14.0. This PR was just too late for being included, but I did label it as "backport-candidate" in case we need to make a new release candidate or would do a bug-fix release.

@pentschev
Copy link
Contributor Author

Thanks @jorisvandenbossche for considering that, we indeed saw that pyarrow 14.0 release is in-process, so we opted for patching conda-forge packages which is probably sufficient to overcome the issues we've been experiencing, we'll probably need a couple more days to confirm that but it seems like a promising solution with minimal trouble for Arrow maintainers. 🙂

@bdice @vyasr @jakirkham please feel free to correct me if needed here.

@bdice
Copy link

bdice commented Oct 26, 2023

Yes, thank you @jorisvandenbossche. I agree with @pentschev that our use case for pyarrow will be satisfied without a backport to 14.x, as long as we have the patch in the conda-forge packages for Arrow 13 which was just merged. We'll want the same patch to carry forward to Arrow 14 on conda-forge, until Arrow 15 is released with the fix upstreamed.👍

However, I see this fixed an issue reported by someone else, so a backport could be very welcome by other users: rstudio/reticulate#1470 (comment)

@vyasr
Copy link
Contributor

vyasr commented Oct 26, 2023

I agree with the above. RAPIDS should be fine with the conda-forge patch and the Arrow 15 bugfix, but it could help other users to get it out sooner.

loicalleyne pushed a commit to loicalleyne/arrow that referenced this pull request Nov 13, 2023
### Rationale for this change

In accordance to apache#38364, we believe that for various reasons (shortening import time, preventing unnecessary resource consumption and potential bugs with S3 library) it is appropriate to avoid initialization of S3 resources at import time and move that step to occur at first-use.

### What changes are included in this PR?

- Remove calls to `ensure_s3_initialized()` that were up until now executed during `import pyarrow.fs`;
- Move `ensure_s3_intialized()` calls to `python/pyarrow/_s3fs.pyx` module;
- Add global flag to mark whether S3 has been previously initialized and `atexit` handlers registered.

### Are these changes tested?

Yes, existing S3 tests check whether it has been initialized, otherwise failing with a C++ exception.

### Are there any user-facing changes?

No, the behavior is now slightly different with S3 initialization not happening immediately after `pyarrow.fs` is imported, but no changes are expected from a user perspective relying on the public API alone.

**This PR contains a "Critical Fix".**
A bug in aws-sdk-cpp reported in aws/aws-sdk-cpp#2681 causes segmentation faults under specific circumstances when Python processes shutdown, specifically observed with Dask+GPUs (so far we were unable to pinpoint the exact correlation of Dask+GPUs+S3). While this definitely doesn't seem to affect all users and is not directly sourced in Arrow, it may affect use cases that are completely independent of S3 to operate, which is particularly problematic in CI where all tests pass successfully but the process crashes at shutdown.
* Closes: apache#38364

Lead-authored-by: Peter Andreas Entschev <[email protected]>
Co-authored-by: Antoine Pitrou <[email protected]>
Signed-off-by: Antoine Pitrou <[email protected]>
loicalleyne pushed a commit to loicalleyne/arrow that referenced this pull request Nov 13, 2023
)

### Rationale for this change

apache#38375 introduced duplicate calls to `ensure_s3_initialized()`.

### What changes are included in this PR?

Deduplicates call to `ensure_s3_initialized()`.

### Are these changes tested?

Yes, covered by existing S3 tests.

### Are there any user-facing changes?

No.

Authored-by: Peter Andreas Entschev <[email protected]>
Signed-off-by: Sutou Kouhei <[email protected]>
raulcd pushed a commit that referenced this pull request Nov 28, 2023
### Rationale for this change

In accordance to #38364, we believe that for various reasons (shortening import time, preventing unnecessary resource consumption and potential bugs with S3 library) it is appropriate to avoid initialization of S3 resources at import time and move that step to occur at first-use.

### What changes are included in this PR?

- Remove calls to `ensure_s3_initialized()` that were up until now executed during `import pyarrow.fs`;
- Move `ensure_s3_intialized()` calls to `python/pyarrow/_s3fs.pyx` module;
- Add global flag to mark whether S3 has been previously initialized and `atexit` handlers registered.

### Are these changes tested?

Yes, existing S3 tests check whether it has been initialized, otherwise failing with a C++ exception.

### Are there any user-facing changes?

No, the behavior is now slightly different with S3 initialization not happening immediately after `pyarrow.fs` is imported, but no changes are expected from a user perspective relying on the public API alone.

**This PR contains a "Critical Fix".**
A bug in aws-sdk-cpp reported in aws/aws-sdk-cpp#2681 causes segmentation faults under specific circumstances when Python processes shutdown, specifically observed with Dask+GPUs (so far we were unable to pinpoint the exact correlation of Dask+GPUs+S3). While this definitely doesn't seem to affect all users and is not directly sourced in Arrow, it may affect use cases that are completely independent of S3 to operate, which is particularly problematic in CI where all tests pass successfully but the process crashes at shutdown.
* Closes: #38364

Lead-authored-by: Peter Andreas Entschev <[email protected]>
Co-authored-by: Antoine Pitrou <[email protected]>
Signed-off-by: Antoine Pitrou <[email protected]>
dgreiss pushed a commit to dgreiss/arrow that referenced this pull request Feb 19, 2024
### Rationale for this change

In accordance to apache#38364, we believe that for various reasons (shortening import time, preventing unnecessary resource consumption and potential bugs with S3 library) it is appropriate to avoid initialization of S3 resources at import time and move that step to occur at first-use.

### What changes are included in this PR?

- Remove calls to `ensure_s3_initialized()` that were up until now executed during `import pyarrow.fs`;
- Move `ensure_s3_intialized()` calls to `python/pyarrow/_s3fs.pyx` module;
- Add global flag to mark whether S3 has been previously initialized and `atexit` handlers registered.

### Are these changes tested?

Yes, existing S3 tests check whether it has been initialized, otherwise failing with a C++ exception.

### Are there any user-facing changes?

No, the behavior is now slightly different with S3 initialization not happening immediately after `pyarrow.fs` is imported, but no changes are expected from a user perspective relying on the public API alone.

**This PR contains a "Critical Fix".**
A bug in aws-sdk-cpp reported in aws/aws-sdk-cpp#2681 causes segmentation faults under specific circumstances when Python processes shutdown, specifically observed with Dask+GPUs (so far we were unable to pinpoint the exact correlation of Dask+GPUs+S3). While this definitely doesn't seem to affect all users and is not directly sourced in Arrow, it may affect use cases that are completely independent of S3 to operate, which is particularly problematic in CI where all tests pass successfully but the process crashes at shutdown.
* Closes: apache#38364

Lead-authored-by: Peter Andreas Entschev <[email protected]>
Co-authored-by: Antoine Pitrou <[email protected]>
Signed-off-by: Antoine Pitrou <[email protected]>
dgreiss pushed a commit to dgreiss/arrow that referenced this pull request Feb 19, 2024
)

### Rationale for this change

apache#38375 introduced duplicate calls to `ensure_s3_initialized()`.

### What changes are included in this PR?

Deduplicates call to `ensure_s3_initialized()`.

### Are these changes tested?

Yes, covered by existing S3 tests.

### Are there any user-facing changes?

No.

Authored-by: Peter Andreas Entschev <[email protected]>
Signed-off-by: Sutou Kouhei <[email protected]>
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.

[Python] Stop initializing s3 upon import
9 participants