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

[data] support configuring retry interval for metadata fetching #43216

Merged
merged 5 commits into from
Feb 21, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 2 additions & 0 deletions python/ray/data/datasource/file_meta_provider.py
Original file line number Diff line number Diff line change
Expand Up @@ -334,6 +334,7 @@ def prefetch_file_metadata(
PARALLELIZE_META_FETCH_THRESHOLD,
RETRY_EXCEPTIONS_FOR_META_FETCH_TASK,
RETRY_MAX_ATTEMPTS_FOR_META_FETCH_TASK,
RETRY_MAX_BACKOFF_S_FOR_META_FETCH_TASK,
_fetch_metadata,
_fetch_metadata_serialization_wrapper,
_SerializedFragment,
Expand All @@ -350,6 +351,7 @@ def fetch_func(fragments):
# Ensure that retry settings are propagated to remote tasks.
retry_match=RETRY_EXCEPTIONS_FOR_META_FETCH_TASK,
retry_max_attempts=RETRY_MAX_ATTEMPTS_FOR_META_FETCH_TASK,
retry_max_interval=RETRY_MAX_BACKOFF_S_FOR_META_FETCH_TASK,
)

return list(
Expand Down
16 changes: 15 additions & 1 deletion python/ray/data/datasource/parquet_datasource.py
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,8 @@
RETRY_EXCEPTIONS_FOR_META_FETCH_TASK = ["AWS Error ACCESS_DENIED", "Timeout"]
# Maximum number of retries for metadata prefetching task due to transient errors.
RETRY_MAX_ATTEMPTS_FOR_META_FETCH_TASK = 32
# Maximum retry back-off interval in seconds for failed metadata prefetching task.
RETRY_MAX_BACKOFF_S_FOR_META_FETCH_TASK = 64

# The number of rows to read per batch. This is sized to generate 10MiB batches
# for rows about 1KiB in size.
Expand Down Expand Up @@ -516,6 +518,7 @@ def _fetch_metadata_serialization_wrapper(
fragments: List[_SerializedFragment],
retry_match: Optional[List[str]],
retry_max_attempts: int,
retry_max_interval: int,
) -> List["pyarrow.parquet.FileMetaData"]:
deserialized_fragments = _deserialize_fragments_with_retry(fragments)
try:
Expand All @@ -524,6 +527,7 @@ def _fetch_metadata_serialization_wrapper(
description="fetch metdata",
match=retry_match,
max_attempts=retry_max_attempts,
max_backoff_s=retry_max_interval,
)
except OSError as e:
raise RuntimeError(
Expand All @@ -536,7 +540,17 @@ def _fetch_metadata_serialization_wrapper(
"```\n"
"ray.data.datasource.parquet_datasource.RETRY_MAX_ATTEMPTS_FOR_META_FETCH_TASK = 64\n" # noqa: E501
"```\n"
"\n"
"To increase the maximum retry backoff interval, configure "
"`RETRY_MAX_BACKOFF_S_FOR_META_FETCH_TASK`. For example:\n"
"```\n"
"ray.data.datasource.parquet_datasource.RETRY_MAX_BACKOFF_S_FOR_META_FETCH_TASK = 128\n" # noqa: E501
"```\n"
"If the error continues to occur, you can also try decresasing the "
"concurency of metadata fetching tasks by setting "
"`NUM_CPUS_FOR_META_FETCH_TASK` to a larger value. For example:\n"
"```\n"
"ray.data.datasource.parquet_datasource.NUM_CPUS_FOR_META_FETCH_TASK = 4.\n" # noqa: E501
"```\n"
"To change which exceptions to retry on, set "
"`RETRY_EXCEPTIONS_FOR_META_FETCH_TASK` to a list of error messages. For "
"example:\n"
Expand Down
Loading