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] Retry on OSError: AWS Error NETWORK_CONNECTION during GetObject operation: curlCode: 28, Timeout was reached #43803

Open
jennifgcrl opened this issue Mar 8, 2024 · 14 comments
Assignees
Labels
bug Something that is supposed to be working; but isn't data Ray Data-related issues P0 Issues that should be fixed in short order

Comments

@jennifgcrl
Copy link

What happened + What you expected to happen

ray.data.read_parquet_bulk crashes on parquet_base_datasource's pq.read_table with

OSError: AWS Error NETWORK_CONNECTION during GetObject operation: curlCode: 28, Timeout was reached

This error should be retried

Versions / Dependencies

ray==2.9.2

Reproduction script

large_dataset = ... # (e.g., s3 directory with 100k parquet files of 1GiB each of arbitrary data)
data = ray.data.read_parquet_bulk(large_dataset)
data.write_datasink(...) # write to s3

Issue Severity

High: It blocks me from completing my task.

@jennifgcrl jennifgcrl added bug Something that is supposed to be working; but isn't triage Needs triage (eg: priority, bug/not-bug, and owning component) labels Mar 8, 2024
@bveeramani
Copy link
Member

Hey @jennifgcrl, could you share a full traceback? Also, what does your cluster look like (type and number of nodes), and how did you choose between read_parquet and read_parquet_bulk?

@meltzerpete
Copy link

Hi there,

I am seeing the same issue and it is a major blocker.

I am using read_parquet for 512 or more parquet chunks at a time and read_binary_files for millions of files. In each case all files are in a common root prefix on S3. I am using about 1000 CPUs, but sometimes more. More CPUs seems to cause the error more often.

Sometimes its error getting information for key, sometimes it's libcurl was given bad argument, sometimes it's timeout like above, however, it's always caused by transient network errors that need to be retried.

I see some fixes for similar problems have been applied in the past, but there are still many gaps.

Some previous related works:

The closest fix is this one: #42027
while it says it fixed for reads, inspecting the codebase in version 2.9.3 it looks like it's only actually fixed for writes.

The error I am seeing is AWS Error NETWORK_CONNECTION which is transient and must be retried. It is on the actual read tasks, as well as metadata tasks. Some gaps are here:

A bit of custom hackery into the codebase has improved the reliability greatly for me, although my solutions are too messy to PR. These code references should hopefully be helpful enough to indicate gaps in the current retry strategies though.

@meltzerpete
Copy link

Here's an example stacktrace with some stuff ***'d out:

Read progress 0: 100%|█████████▉| 3711/3712 [00:39<00:00, 49.63it/s]Traceback (most recent call last):
  File "/tmp/ray/session_2024-03-05_11-09-55_961367_273/runtime_resources/working_dir_files/_ray_pkg_ddef4fad01df9951/src/my_file.py", line 277, in <module>
    ***
  File "/tmp/ray/session_2024-03-05_11-09-55_961367_273/runtime_resources/working_dir_files/_ray_pkg_ddef4fad01df9951/src/my_file.py", line 273, in ***
  File "/tmp/ray/session_2024-03-05_11-09-55_961367_273/runtime_resources/working_dir_files/_ray_pkg_ddef4fad01df9951/src/my_file.py", line 152, in ***
    print('size of prev_batches:', prev_batches.count())
  File "/home/ray/anaconda3/lib/python3.10/site-packages/ray/data/dataset.py", line 2606, in count
    [get_num_rows.remote(block) for block in self.get_internal_block_refs()]
  File "/home/ray/anaconda3/lib/python3.10/site-packages/ray/data/dataset.py", line 4779, in get_internal_block_refs
    blocks = self._plan.execute().get_blocks()
  File "/home/ray/anaconda3/lib/python3.10/site-packages/ray/data/_internal/lazy_block_list.py", line 293, in get_blocks
    blocks, _ = self._get_blocks_with_metadata()
  File "/home/ray/anaconda3/lib/python3.10/site-packages/ray/data/_internal/lazy_block_list.py", line 327, in _get_blocks_with_metadata
    meta = ray.get(refs_list.pop(-1))
  File "/home/ray/anaconda3/lib/python3.10/site-packages/ray/_private/auto_init_hook.py", line 22, in auto_init_wrapper
    return fn(*args, **kwargs)
  File "/home/ray/anaconda3/lib/python3.10/site-packages/ray/_private/client_mode_hook.py", line 103, in wrapper
    return func(*args, **kwargs)
  File "/home/ray/anaconda3/lib/python3.10/site-packages/ray/_private/worker.py", line 2624, in get
    raise value.as_instanceof_cause()
ray.exceptions.RayTaskError(OSError): �[36mray::_execute_read_task_split()�[39m (pid=25144, ip=10.83.226.216)
  File "/home/ray/anaconda3/lib/python3.10/site-packages/ray/data/_internal/lazy_block_list.py", line 637, in _execute_read_task_split
    for block in blocks:
  File "/home/ray/anaconda3/lib/python3.10/site-packages/ray/data/datasource/datasource.py", line 237, in __call__
    yield from result
  File "/home/ray/anaconda3/lib/python3.10/site-packages/ray/data/_internal/execution/operators/map_transformer.py", line 430, in __call__
    for block in blocks:
  File "/home/ray/anaconda3/lib/python3.10/site-packages/ray/data/_internal/execution/operators/map_transformer.py", line 371, in __call__
    for data in iter:
  File "/home/ray/anaconda3/lib/python3.10/site-packages/ray/data/datasource/parquet_datasource.py", line 508, in _read_fragments
    for batch in batches:
  File "pyarrow/_dataset.pyx", line 3414, in _iterator
  File "pyarrow/_dataset.pyx", line 3032, in pyarrow._dataset.TaggedRecordBatchIterator.__next__
  File "pyarrow/error.pxi", line 144, in pyarrow.lib.pyarrow_internal_check_status
  File "pyarrow/error.pxi", line 115, in pyarrow.lib.check_status
OSError: When reading information for key '***_000000.parquet' in bucket 'MY_BUCKET': AWS Error NETWORK_CONNECTION during HeadObject operation: curlCode: 28, Timeout was reached

@bveeramani bveeramani added P1 Issue that should be fixed within a few weeks data Ray Data-related issues and removed triage Needs triage (eg: priority, bug/not-bug, and owning component) labels Mar 8, 2024
@Sri-nidhi
Copy link

@bveeramani May I know when we can expect a fix on this, Im facing the same issue when im trying to read around 42k parquet files from s3 directory

@ronyw7
Copy link
Contributor

ronyw7 commented Apr 2, 2024

We are seeing similar issues when reading ~1M images from a S3 bucket with read_images, which causes e2e training to end abruptly.

- ReadImage->Map(wnid_to_index): 4 active, 360 queued, [cpu: 4.0, objects: 673.3MB]:  67%|██████▋   | 1463/2188 [50:34<23:33,  1.95s/it]

- ReadImage->Map(wnid_to_index): 4 active, 360 queued, [cpu: 4.0, objects: 673.3MB]: 100%|██████████| 1463/1463 [50:34<00:00,  1.95s/it]

Traceback (most recent call last):
ray.exceptions.RayTaskError(OSError): �[36mray::ReadImage->Map(wnid_to_index)()�[39m (pid=257328, ip=10.0.34.58)
    for b_out in map_transformer.apply_transform(iter(blocks), ctx):
  File "/home/ubuntu/miniconda3/envs/ray-gpu/lib/python3.10/site-packages/ray/data/_internal/execution/operators/map_transformer.py", line 392, in __call__
    for data in iter:
  File "/home/ubuntu/miniconda3/envs/ray-gpu/lib/python3.10/site-packages/ray/data/_internal/execution/operators/map_transformer.py", line 134, in _udf_timed_iter
    output = next(input)
  File "/home/ubuntu/miniconda3/envs/ray-gpu/lib/python3.10/site-packages/ray/data/_internal/execution/operators/map_transformer.py", line 216, in __call__
    yield from self._row_fn(input, ctx)
  File "/home/ubuntu/miniconda3/envs/ray-gpu/lib/python3.10/site-packages/ray/data/_internal/planner/plan_udf_map_op.py", line 239, in transform_fn
    for row in rows:
  File "/home/ubuntu/miniconda3/envs/ray-gpu/lib/python3.10/site-packages/ray/data/_internal/execution/operators/map_transformer.py", line 269, in __call__
    for block in blocks:
  File "/home/ubuntu/miniconda3/envs/ray-gpu/lib/python3.10/site-packages/ray/data/_internal/execution/operators/map_transformer.py", line 253, in __call__
    yield from self._block_fn(input, ctx)
  File "/home/ubuntu/miniconda3/envs/ray-gpu/lib/python3.10/site-packages/ray/data/_internal/planner/plan_read_op.py", line 91, in do_read
    yield from call_with_retry(
  File "/home/ubuntu/miniconda3/envs/ray-gpu/lib/python3.10/site-packages/ray/data/datasource/datasource.py", line 164, in __call__
    yield from result
  File "/home/ubuntu/miniconda3/envs/ray-gpu/lib/python3.10/site-packages/ray/data/datasource/file_based_datasource.py", line 258, in read_task_fn
    yield from make_async_gen(
  File "/home/ubuntu/miniconda3/envs/ray-gpu/lib/python3.10/site-packages/ray/data/_internal/util.py", line 941, in make_async_gen
    raise next_item
  File "/home/ubuntu/miniconda3/envs/ray-gpu/lib/python3.10/site-packages/ray/data/_internal/util.py", line 918, in execute_computation
    for item in fn(thread_safe_generator):
  File "/home/ubuntu/miniconda3/envs/ray-gpu/lib/python3.10/site-packages/ray/data/datasource/file_based_datasource.py", line 232, in read_files
    with _open_file_with_retry(
  File "/home/ubuntu/miniconda3/envs/ray-gpu/lib/python3.10/site-packages/ray/data/datasource/file_based_datasource.py", line 522, in _open_file_with_retry
    return call_with_retry(
  File "/home/ubuntu/miniconda3/envs/ray-gpu/lib/python3.10/site-packages/ray/data/_internal/util.py", line 995, in call_with_retry
    raise e from None
  File "/home/ubuntu/miniconda3/envs/ray-gpu/lib/python3.10/site-packages/ray/data/_internal/util.py", line 982, in call_with_retry
    return f()
  File "/home/ubuntu/miniconda3/envs/ray-gpu/lib/python3.10/site-packages/ray/data/datasource/file_based_datasource.py", line 234, in <lambda>
    lambda: open_input_source(fs, read_path, **open_stream_args),
  File "/home/ubuntu/miniconda3/envs/ray-gpu/lib/python3.10/site-packages/ray/data/datasource/file_based_datasource.py", line 344, in _open_input_source
    file = filesystem.open_input_stream(path, buffer_size=buffer_size, **open_args)
  File "pyarrow/_fs.pyx", line 822, in pyarrow._fs.FileSystem.open_input_stream
  File "pyarrow/error.pxi", line 154, in pyarrow.lib.pyarrow_internal_check_status
  File "pyarrow/error.pxi", line 91, in pyarrow.lib.check_status
OSError: When reading information for key 'IMAGE_KEY' in bucket 'MY_BUCKET': AWS Error NETWORK_CONNECTION during HeadObject operation: curlCode: 28, Timeout was reached

@murthyn
Copy link

murthyn commented Apr 9, 2024

Is there any update on fixing this? Also running into this and it is of high severity for me as well

@anyscalesam anyscalesam added P0 Issues that should be fixed in short order and removed P1 Issue that should be fixed within a few weeks labels Apr 9, 2024
@anyscalesam
Copy link
Collaborator

@murthyn - this is high priority for us but we're a little swamped; it's funded as part of our planning through May. Balaji will provide more details as he has them.

@anyscalesam
Copy link
Collaborator

re-reading this - @meltzerpete do you think you'd be game to contribute a PR; we can pair with someone to help shepherd through on the Anyscale side.

Your breakdown of the problem and whereabouts to solve is quite spectacular :) cc @c21

@raulchen
Copy link
Contributor

raulchen commented May 2, 2024

hi @jennifgcrl , looks like this is a transient error.
For ray 2.10+, we now support automatic retry of these kind of errors.
Could you try again with the latest Ray?

@anyscalesam
Copy link
Collaborator

tag @murthyn @ronyw7 @Sri-nidhi @meltzerpete as well see @raulchen above^

@meltzerpete
Copy link

tag @murthyn @ronyw7 @Sri-nidhi @meltzerpete as well see @raulchen above^

Thanks for the updates. Hopefully this has been solved. I'm unavailable until Tuesday, but will test this then. Failing that happy to try and support with PR if there's someone to pair with.

@anyscalesam anyscalesam added the @external-author-action-required Alternate tag for PRs where the author doesn't have labeling permission. label May 3, 2024
@bveeramani bveeramani self-assigned this May 6, 2024
@meltzerpete
Copy link

@raulchen @anyscalesam

I'm pretty sure I tried ray 2.10 when it was first released and found the issue persisted, however, I have tested today with ray 20.0.0 and can confirm the issue appears to be solved.

I am seeing no transient errors at all, thanks so much!

@meltzerpete
Copy link

Ah in fact perhaps I spoke too soon 😬

I am seeing this:

failed to write s3://*****: ray::ReadParquet() (pid=1061231, ip=10.83.226.188)
    for b_out in map_transformer.apply_transform(iter(blocks), ctx):
  File "/home/ray/anaconda3/lib/python3.10/site-packages/ray/data/_internal/execution/operators/map_transformer.py", line 392, in __call__
    for data in iter:
  File "/home/ray/anaconda3/lib/python3.10/site-packages/ray/data/_internal/execution/operators/map_transformer.py", line 253, in __call__
    yield from self._block_fn(input, ctx)
  File "/home/ray/anaconda3/lib/python3.10/site-packages/ray/data/_internal/planner/plan_read_op.py", line 91, in do_read
    yield from call_with_retry(
  File "/home/ray/anaconda3/lib/python3.10/site-packages/ray/data/datasource/datasource.py", line 163, in __call__
    yield from result
  File "/home/ray/anaconda3/lib/python3.10/site-packages/ray/data/datasource/parquet_datasource.py", line 549, in _read_fragments
    for batch in batches:
  File "pyarrow/_dataset.pyx", line 3769, in _iterator
  File "pyarrow/_dataset.pyx", line 3387, in pyarrow._dataset.TaggedRecordBatchIterator.__next__
  File "pyarrow/error.pxi", line 154, in pyarrow.lib.pyarrow_internal_check_status
  File "pyarrow/error.pxi", line 91, in pyarrow.lib.check_status
OSError: When reading information for key '*****' in bucket '*****': AWS Error NETWORK_CONNECTION during HeadObject operation: curlCode: 28, Timeout was reached

I'm seeing this several times, but this looks like the only place it's erroring. Before there were many.

@anyscalesam anyscalesam removed the @external-author-action-required Alternate tag for PRs where the author doesn't have labeling permission. label May 9, 2024
@DimitarSirakov
Copy link

Is there any update ? I'm facing the same issue...

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something that is supposed to be working; but isn't data Ray Data-related issues P0 Issues that should be fixed in short order
Projects
None yet
Development

No branches or pull requests

9 participants