ray-project / ray

Ray is a unified framework for scaling AI and Python applications. Ray consists of a core distributed runtime and a set of AI Libraries for accelerating ML workloads.
https://ray.io
Apache License 2.0
32.16k stars 5.48k forks source link

[Core] Excessive inlined arguments cause `rpc::TaskSpec` to exceed the gRPC request size limit. #45685

Open Catch-Bull opened 1 month ago

Catch-Bull commented 1 month ago

What happened + What you expected to happen

Ray supports controlling the total size of inlined task arguments within a specified range through config task_rpc_inlined_bytes_limit, but when the arguments is inlined returns value, config task_rpc_inlined_bytes_limit not work.

Versions / Dependencies

ray: 2.23.0 python: 3.10.14

Reproduction script

test.py:

import ray; ray.init()
@ray.remote
def func1():
    return b'0' * 5 * 1024 * 1024

@ray.remote
def func2(*args):
    s = 0
    for arg in args:
        s += len(arg)
    return s

refs = []
for i in range(300):
    refs.append(func1.remote())
print("size: ", ray.get(func2.remote(*refs)))

run command:

RAY_BACKEND_LOG_LEVEL=debug RAY_max_direct_call_object_size=10240000 python test.py

this script will hang forever, and found this error on python-core-driver-01000000ffffffffffffffffffffffffffffffffffffffffffffffff_xxxx.log:

[2024-06-03 21:18:57,341 D 3559016 3559683] metric_exporter.cc:98: Exporting metrics, metrics: 43, payload size: 9394
[2024-06-03 21:18:57,481 I 3559016 3559685] raylet_client.cc:368: Error reporting task backlog information: GrpcUnknown: RPC Error message: Sent message larger than max (1572875574 vs. 536870912); RPC Error details: 
[2024-06-03 21:18:57,482 W 3559016 3559685] direct_task_transport.cc:558: The worker failed to receive a response from the local raylet, but raylet is still alive. Try again on a local node. Error: GrpcUnknown: RPC Error message: Sent message larger than max (1572875540 vs. 536870912); RPC Error details: 
[2024-06-03 21:18:57,853 D 3559016 3560532] accessor.cc:751: Accessor added task events grpc OK
[2024-06-03 21:18:58,087 D 3559016 3559685] direct_task_transport.cc:405: Requesting lease from raylet deac14b134d7f0546c61394e2c57bf5fe727e6b7a8f03a802abae1b4 for task 8ce23775a55033a85eebb7d462edb501f8c98b0c01000000
[2024-06-03 21:18:58,853 D 3559016 3560532] accessor.cc:751: Accessor added task events grpc OK
[2024-06-03 21:18:59,853 D 3559016 3560532] accessor.cc:751: Accessor added task events grpc OK
[2024-06-03 21:19:00,792 W 3559016 3559685] direct_task_transport.cc:558: The worker failed to receive a response from the local raylet, but raylet is still alive. Try again on a local node. Error: GrpcUnknown: RPC Error message: Sent message larger than max (1572875540 vs. 536870912); RPC Error details: 
[2024-06-03 21:19:00,854 D 3559016 3560532] accessor.cc:751: Accessor added task events grpc OK
[2024-06-03 21:19:01,544 D 3559016 3559685] direct_task_transport.cc:405: Requesting lease from raylet deac14b134d7f0546c61394e2c57bf5fe727e6b7a8f03a802abae1b4 for task c232dfd95056e673fbede8e09d592fb980537f1f01000000
[2024-06-03 21:19:01,854 D 3559016 3560532] accessor.cc:751: Accessor added task events grpc OK
[2024-06-03 21:19:02,381 I 3559016 3559685] raylet_client.cc:368: Error reporting task backlog information: GrpcUnknown: RPC Error message: Sent message larger than max (1572875574 vs. 536870912); RPC Error details: 
[2024-06-03 21:19:02,381 W 3559016 3559685] direct_task_transport.cc:558: The worker failed to receive a response from the local raylet, but raylet is still alive. Try again on a local node. Error: GrpcUnknown: RPC Error message: Sent message larger than max (1572875540 vs. 536870912); RPC Error details: 
[2024-06-03 21:19:02,854 D 3559016 3560532] accessor.cc:751: Accessor added task events grpc OK
[2024-06-03 21:19:03,132 D 3559016 3559685] direct_task_transport.cc:405: Requesting lease from raylet deac14b134d7f0546c61394e2c57bf5fe727e6b7a8f03a802abae1b4 for task 2eb25a531574fecd5d55117a6f1377c8ae579c8301000000
[2024-06-03 21:19:03,854 D 3559016 3560532] accessor.cc:751: Accessor added task events grpc OK

Issue Severity

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

Catch-Bull commented 1 month ago

@jjyao I think it is necessary to discussion on whether this is a bug. If this is a bug, we need to modify the function InlineDependencies: to put arguments into plasma when the inlined arguments exceed task_rpc_inlined_bytes_limit, but it seems not easy.

jjyao commented 1 month ago

@Catch-Bull I think it's a known limitation: #7269. Our workaround was to increase grpc request size limit so that in most cases, the sum of inlined arguments won't exceed the limit.

Agree that the fix you suggested is not easy.