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

[Python] Stop initializing s3 upon import #38364

Closed
vyasr opened this issue Oct 19, 2023 · 6 comments · Fixed by #38375
Closed

[Python] Stop initializing s3 upon import #38364

vyasr opened this issue Oct 19, 2023 · 6 comments · Fixed by #38375
Assignees
Labels
Component: C++ Component: Python Critical Fix Bugfixes for security vulnerabilities, crashes, or invalid data. Type: enhancement
Milestone

Comments

@vyasr
Copy link
Contributor

vyasr commented Oct 19, 2023

Describe the enhancement requested

Currently pyarrow initializes the s3 filesystem when pyarrow.fs is imported. This leads to AWS consuming resources on startup that may never be used if the user is not actually taking advantage of that support. Ideally the s3fs would instead be delayed to first use to avoid AWS spinning up unnecessary threads/doing work on pyarrow import.

Making this change would also allow sidestepping a bug present in newer versions of the aws-sdk-cpp that occasionally leads to segfaults simply by using the AWS APIs, at least for the majority of users who are not using the s3fs by default.

Component(s)

C++, Python

@vyasr
Copy link
Contributor Author

vyasr commented Oct 19, 2023

I believe @pentschev has put together a patch demonstrating how this could be done.

@jorisvandenbossche jorisvandenbossche changed the title Stop initializing s3 upon import [Python] Stop initializing s3 upon import Oct 20, 2023
@felipecrv
Copy link
Contributor

@westonpace has worked on the problem of S3 initialization in the past and it seems like we can either init eagerly and avoid some issues or init lazily and suffer bugs. Any solution here needs to consider the issues that lead to the eager init solution.

@felipecrv
Copy link
Contributor

Reference #33858

@jakirkham
Copy link

If Weston is able to review PR ( #38375 ), we would appreciate hearing his insights 🙂

pitrou added a commit that referenced this issue Oct 24, 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 <peter@entschev.com>
Co-authored-by: Antoine Pitrou <pitrou@free.fr>
Signed-off-by: Antoine Pitrou <antoine@python.org>
@pitrou pitrou added this to the 15.0.0 milestone Oct 24, 2023
@pitrou
Copy link
Member

pitrou commented Oct 24, 2023

@pentschev Can you post a message on this issue, so that you can assign it to you?

@pentschev
Copy link
Contributor

Sure, please assign the issue to myself, @pitrou .

rapids-bot bot pushed a commit to rapidsai/cudf that referenced this issue Oct 24, 2023
The aws-sdk-pinning is proving to be far too problematic to maintain. It causes conflicts in many environments due to its common usage across many other packages in the conda-forge ecosystem that have since updated their pinning to require newer versions than the 1.10.* that we've pinned to. This reversion will unblock most of RAPIDS CI. We will search for alternative fixes to the dask-cuda/distributed issues that we're observing (in particular, resolution of the underlying issues apache/arrow#38364 and aws/aws-sdk-cpp#2681).

Authors:
  - Vyas Ramasubramani (https://github.com/vyasr)

Approvers:
  - Ray Douglass (https://github.com/raydouglass)
  - Bradley Dice (https://github.com/bdice)
  - Lawrence Mitchell (https://github.com/wence-)
  - Peter Andreas Entschev (https://github.com/pentschev)

URL: #14319
JerAguilon pushed a commit to JerAguilon/arrow that referenced this issue 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 <peter@entschev.com>
Co-authored-by: Antoine Pitrou <pitrou@free.fr>
Signed-off-by: Antoine Pitrou <antoine@python.org>
loicalleyne pushed a commit to loicalleyne/arrow that referenced this issue 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 <peter@entschev.com>
Co-authored-by: Antoine Pitrou <pitrou@free.fr>
Signed-off-by: Antoine Pitrou <antoine@python.org>
@raulcd raulcd modified the milestones: 15.0.0, 14.0.2 Nov 28, 2023
raulcd pushed a commit that referenced this issue 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 <peter@entschev.com>
Co-authored-by: Antoine Pitrou <pitrou@free.fr>
Signed-off-by: Antoine Pitrou <antoine@python.org>
@amoeba amoeba added the Critical Fix Bugfixes for security vulnerabilities, crashes, or invalid data. label Dec 21, 2023
dgreiss pushed a commit to dgreiss/arrow that referenced this issue 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 <peter@entschev.com>
Co-authored-by: Antoine Pitrou <pitrou@free.fr>
Signed-off-by: Antoine Pitrou <antoine@python.org>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Component: C++ Component: Python Critical Fix Bugfixes for security vulnerabilities, crashes, or invalid data. Type: enhancement
Projects
None yet
Development

Successfully merging a pull request may close this issue.

8 participants