Skip to content

Commit

Permalink
GH-38364: [Python] Initialize S3 on first use (#38375)
Browse files Browse the repository at this point in the history
### 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>
  • Loading branch information
2 people authored and raulcd committed Nov 28, 2023
1 parent ba53748 commit 4ed68c0
Show file tree
Hide file tree
Showing 3 changed files with 18 additions and 3 deletions.
11 changes: 11 additions & 0 deletions python/pyarrow/_s3fs.pyx
Expand Up @@ -70,6 +70,13 @@ def finalize_s3():
check_status(CFinalizeS3())


def ensure_s3_finalized():
"""
Finalize S3 if already initialized
"""
check_status(CEnsureS3Finalized())


def resolve_s3_region(bucket):
"""
Resolve the S3 region of a bucket.
Expand All @@ -93,6 +100,8 @@ def resolve_s3_region(bucket):
c_string c_bucket
c_string c_region

ensure_s3_initialized()

c_bucket = tobytes(bucket)
with nogil:
c_region = GetResultValue(ResolveS3BucketRegion(c_bucket))
Expand Down Expand Up @@ -260,6 +269,8 @@ cdef class S3FileSystem(FileSystem):
load_frequency=900, proxy_options=None,
allow_bucket_creation=False, allow_bucket_deletion=False,
retry_strategy: S3RetryStrategy = AwsStandardS3RetryStrategy(max_attempts=3)):
ensure_s3_initialized()

cdef:
optional[CS3Options] options
shared_ptr[CS3FileSystem] wrapped
Expand Down
9 changes: 6 additions & 3 deletions python/pyarrow/fs.py
Expand Up @@ -54,13 +54,16 @@
from pyarrow._s3fs import ( # noqa
AwsDefaultS3RetryStrategy, AwsStandardS3RetryStrategy,
S3FileSystem, S3LogLevel, S3RetryStrategy, ensure_s3_initialized,
finalize_s3, initialize_s3, resolve_s3_region)
finalize_s3, ensure_s3_finalized, initialize_s3, resolve_s3_region)
except ImportError:
_not_imported.append("S3FileSystem")
else:
ensure_s3_initialized()
# GH-38364: we don't initialize S3 eagerly as that could lead
# to crashes at shutdown even when S3 isn't used.
# Instead, S3 is initialized lazily using `ensure_s3_initialized`
# in assorted places.
import atexit
atexit.register(finalize_s3)
atexit.register(ensure_s3_finalized)


def __getattr__(name):
Expand Down
1 change: 1 addition & 0 deletions python/pyarrow/includes/libarrow_fs.pxd
Expand Up @@ -211,6 +211,7 @@ cdef extern from "arrow/filesystem/api.h" namespace "arrow::fs" nogil:
const CS3GlobalOptions& options)
cdef CStatus CEnsureS3Initialized "arrow::fs::EnsureS3Initialized"()
cdef CStatus CFinalizeS3 "arrow::fs::FinalizeS3"()
cdef CStatus CEnsureS3Finalized "arrow::fs::EnsureS3Finalized"()

cdef CResult[c_string] ResolveS3BucketRegion(const c_string& bucket)

Expand Down

0 comments on commit 4ed68c0

Please sign in to comment.