This is an automated email from the ASF dual-hosted git repository.
apitrou pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/arrow.git
The following commit(s) were added to refs/heads/main by this push:
new 7751444be2 GH-38364: [Python] Initialize S3 on first use (#38375)
7751444be2 is described below
commit 7751444be23a433d73d0722d34d5107ca35a714f
Author: Peter Andreas Entschev <[email protected]>
AuthorDate: Tue Oct 24 17:42:06 2023 +0200
GH-38364: [Python] Initialize S3 on first use (#38375)
### Rationale for this change
In accordance to https://github.com/apache/arrow/issues/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
https://github.com/aws/aws-sdk-cpp/issues/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 wh [...]
* Closes: #38364
Lead-authored-by: Peter Andreas Entschev <[email protected]>
Co-authored-by: Antoine Pitrou <[email protected]>
Signed-off-by: Antoine Pitrou <[email protected]>
---
python/pyarrow/_s3fs.pyx | 11 +++++++++++
python/pyarrow/fs.py | 9 ++++++---
python/pyarrow/includes/libarrow_fs.pxd | 1 +
3 files changed, 18 insertions(+), 3 deletions(-)
diff --git a/python/pyarrow/_s3fs.pyx b/python/pyarrow/_s3fs.pyx
index ab45171369..f7f2a5f808 100644
--- a/python/pyarrow/_s3fs.pyx
+++ b/python/pyarrow/_s3fs.pyx
@@ -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.
@@ -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))
@@ -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
diff --git a/python/pyarrow/fs.py b/python/pyarrow/fs.py
index 36655c7d12..ead750ca44 100644
--- a/python/pyarrow/fs.py
+++ b/python/pyarrow/fs.py
@@ -54,13 +54,16 @@ try:
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):
diff --git a/python/pyarrow/includes/libarrow_fs.pxd
b/python/pyarrow/includes/libarrow_fs.pxd
index 2727fc2011..cb30f4e750 100644
--- a/python/pyarrow/includes/libarrow_fs.pxd
+++ b/python/pyarrow/includes/libarrow_fs.pxd
@@ -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)