ray-project / ray

Ray is an AI compute engine. Ray consists of a core distributed runtime and a set of AI Libraries for accelerating ML workloads.
https://ray.io
Apache License 2.0
34.16k stars 5.8k forks source link

[Core] Error in external storage writing for object spilling #33913

Open joanna-yoo opened 1 year ago

joanna-yoo commented 1 year ago

What happened + What you expected to happen

I'm running ray serve with k8, and I set up the object spilling config as follows:

containers:
  - name: ray-worker
       env:
          - name: "RAY_object_spilling_config"
            value: "{\"type\":\"smart_open\",\"params\":{\"uri\":\"gs://some/bucket/path\"}}"

But this is the error that I got for all of the writes.

Traceback (most recent call last):
  File "python/ray/_raylet.pyx", line 1249, in ray._raylet.spill_objects_handler
  File "python/ray/_raylet.pyx", line 1252, in ray._raylet.spill_objects_handler
  File "//.pyenv/versions/3.10.4/lib/python3.10/site-packages/ray/_private/external_storage.py", line 668, in spill_objects
    return _external_storage.spill_objects(object_refs, owner_addresses)
  File "//.pyenv/versions/3.10.4/lib/python3.10/site-packages/ray/_private/external_storage.py", line 541, in spill_objects
    return self._write_multiple_objects(
  File "//.pyenv/versions/3.10.4/lib/python3.10/site-packages/ray/_private/external_storage.py", line 150, in _write_multiple_objects
    assert written_bytes == payload_len

Under which circumstances would this be false?

Versions / Dependencies

Docker base image: rayproject/ray:2.2.0-py310-gpu ray[serve]==2.2.0

Reproduction script

I cannot reproduce it in dev setup unfortunately. : ( This works fine locally.

import ray
import smart_open
from ray._private import external_storage

buf = b'0' * 1024 * 1024
object_ref = ray.put(buf)
f = smart_open.open('gs://some/path/test.pkl'), 'wb')
e = external_storage.ExternalStorageSmartOpenImpl('gs://some/path')
e._write_multiple_objects(f, [object_ref], [b'test_address'], 'gs://some/path')

Issue Severity

Medium: It is a significant difficulty but I can work around it.

rkooo567 commented 1 year ago

Hmm @franklsf95 have you seen any similar issue from shuffle workloads? do you guys use S3 for spilling?

frank-lsf commented 1 year ago

@rkooo567 Yes I think we've seen this when we were stress testing shuffle. In the end we opted to use local SSD instead of S3/GCS for shuffle because of this. For example, we couldn't get 1TB shuffle to work reliably on S3 due to all sorts of failures, and this is one that we have seen.

Although, I thought we were deprecating smart_open in favor of the PyArrow file systems? I remember this problem is gone with the Arrow implementation. Does Arrow support GCS yet?

rkooo567 commented 1 year ago

I believe it supports (dataset already supports GCS iirc). I think this feature still needs more productionization (it is also not well tested in CI). There may be a chance this can be fixed sooner or later..

rkooo567 commented 1 year ago

@joanna-yoo we recommend you to just use the disk based spilling, and if you need high throughput, consider mounting faster local SSD (e.g., NVME coming with i3 instances).