Skip to content

[Core] The actor task hangs when it is re-submitted #46538

@Catch-Bull

Description

@Catch-Bull

What happened + What you expected to happen

During the actor task resubmit, Ray assumes that failures in RPC PushActorTask caused by network issues are due to the ACTOR_DIED. However, we have found that there are instances where RPC failures are caused by the interruption of the TCP connection, and the actor is still running. This results in the resubmitted actor tasks always canceled by actor, and got log like this: [2024-07-10 20:17:00,616 E 1138212 1138212] actor_scheduling_queue.cc:135: Cancelling stale RPC with seqno 2026 < 2027

Similar issues: #46301

Versions / Dependencies

ray master, commit: ed8d52e

Reproduction script

test.py:

import ray
ray.init()

@ray.remote
class Actor:
    def run(self, func):
        return func(self)

def func(self):
    return True

actor = Actor.options(max_task_retries=-1).remote()

while True:
    import time
    time.sleep(0.1)
    ray.get(actor.run.remote(func))
    print("done")
  1. we run test.py first:
RAY_BACKEND_LOG_LEVEL=debug python test.py
  1. we got actor CoreWorkerServer gRPC server port from it's core worker log, like this:
[2024-07-10 20:12:03,881 I 1138212 1138212] grpc_server.cc:129: worker server started, listening on port 27655.
  1. kill tcp connection:
watch -n 0.01 sudo ss --kill --tcp src :27655 

we will got log like this, and actor task will hang:

done
done
done
done
done
done
done
done
done
done
done
done
done
(Actor pid=1252131) [2024-07-10 21:07:50,698 E 1252131 1252131] actor_scheduling_queue.cc:135: Cancelling stale RPC with seqno 478 < 479
(raylet) [2024-07-10 21:07:01,033 I 1252131 1252131] logging.cc:293: Set ray log level from environment variable RAY_BACKEND_LOG_LEVEL to -1 [repeated 6x across cluster] (Ray deduplicates logs by default. Set RAY_DEDUP_LOGS=0 to disable log deduplication, or see https://docs.ray.io/en/master/ray-observability/user-guides/configure-logging.html#log-deduplication for more options.)
(Actor pid=1252131) [2024-07-10 21:07:51,698 E 1252131 1252131] actor_scheduling_queue.cc:135: Cancelling stale RPC with seqno 478 < 479
(Actor pid=1252131) [2024-07-10 21:07:52,698 E 1252131 1252131] actor_scheduling_queue.cc:135: Cancelling stale RPC with seqno 478 < 479
(Actor pid=1252131) [2024-07-10 21:07:53,698 E 1252131 1252131] actor_scheduling_queue.cc:135: Cancelling stale RPC with seqno 478 < 479
(Actor pid=1252131) [2024-07-10 21:07:54,699 E 1252131 1252131] actor_scheduling_queue.cc:135: Cancelling stale RPC with seqno 478 < 479
(Actor pid=1252131) [2024-07-10 21:07:55,699 E 1252131 1252131] actor_scheduling_queue.cc:135: Cancelling stale RPC with seqno 478 < 479
(Actor pid=1252131) [2024-07-10 21:07:56,699 E 1252131 1252131] actor_scheduling_queue.cc:135: Cancelling stale RPC with seqno 478 < 479
(Actor pid=1252131) [2024-07-10 21:07:57,699 E 1252131 1252131] actor_scheduling_queue.cc:135: Cancelling stale RPC with seqno 478 < 479
(Actor pid=1252131) [2024-07-10 21:07:58,700 E 1252131 1252131] actor_scheduling_queue.cc:135: Cancelling stale RPC with seqno 478 < 479
(Actor pid=1252131) [2024-07-10 21:07:59,700 E 1252131 1252131] actor_scheduling_queue.cc:135: Cancelling stale RPC with seqno 478 < 479
(Actor pid=1252131) [2024-07-10 21:08:00,700 E 1252131 1252131] actor_scheduling_queue.cc:135: Cancelling stale RPC with seqno 478 < 479
(Actor pid=1252131) [2024-07-10 21:08:01,700 E 1252131 1252131] actor_scheduling_queue.cc:135: Cancelling stale RPC with seqno 478 < 479
(Actor pid=1252131) [2024-07-10 21:08:02,701 E 1252131 1252131] actor_scheduling_queue.cc:135: Cancelling stale RPC with seqno 478 < 479
(Actor pid=1252131) [2024-07-10 21:08:03,701 E 1252131 1252131] actor_scheduling_queue.cc:135: Cancelling stale RPC with seqno 478 < 479

Issue Severity

None

Metadata

Metadata

Labels

P1Issue that should be fixed within a few weeksbugSomething that is supposed to be working; but isn'tcoreIssues that should be addressed in Ray Core

Type

No type

Projects

No projects

Milestone

No milestone

Relationships

None yet

Development

No branches or pull requests

Issue actions