mirror of
https://github.com/vale981/ray
synced 2025-03-10 21:36:39 -04:00
[Core] Fix ray start failure to due to bug of redis address detection (#11735)
* Fix ray start failure to due redis address detection bug * Address comment
This commit is contained in:
parent
53aac55739
commit
31598338b3
2 changed files with 21 additions and 10 deletions
|
@ -134,16 +134,20 @@ def find_redis_address(address=None):
|
|||
# the first argument.
|
||||
# Explanation: https://unix.stackexchange.com/a/432681
|
||||
# More info: https://github.com/giampaolo/psutil/issues/1179
|
||||
for arglist in proc.cmdline():
|
||||
# Given we're merely seeking --redis-address, we just split
|
||||
# every argument on spaces for now.
|
||||
for arg in arglist.split(" "):
|
||||
# TODO(ekl): Find a robust solution for locating Redis.
|
||||
if arg.startswith("--redis-address="):
|
||||
proc_addr = arg.split("=")[1]
|
||||
if address is not None and address != proc_addr:
|
||||
continue
|
||||
redis_addresses.add(proc_addr)
|
||||
cmdline = proc.cmdline()
|
||||
# NOTE(kfstorm): To support Windows, we can't use
|
||||
# `os.path.basename(cmdline[0]) == "raylet"` here.
|
||||
if len(cmdline) > 0 and "raylet" in os.path.basename(cmdline[0]):
|
||||
for arglist in cmdline:
|
||||
# Given we're merely seeking --redis-address, we just split
|
||||
# every argument on spaces for now.
|
||||
for arg in arglist.split(" "):
|
||||
# TODO(ekl): Find a robust solution for locating Redis.
|
||||
if arg.startswith("--redis-address="):
|
||||
proc_addr = arg.split("=")[1]
|
||||
if address is not None and address != proc_addr:
|
||||
continue
|
||||
redis_addresses.add(proc_addr)
|
||||
except psutil.AccessDenied:
|
||||
pass
|
||||
except psutil.NoSuchProcess:
|
||||
|
|
|
@ -18,6 +18,7 @@ import ray.cluster_utils
|
|||
import ray.test_utils
|
||||
from ray import resource_spec
|
||||
import setproctitle
|
||||
import subprocess
|
||||
|
||||
from ray.test_utils import (check_call_ray, RayTestTimeoutException,
|
||||
wait_for_condition, wait_for_num_actors)
|
||||
|
@ -522,6 +523,12 @@ def test_export_after_shutdown(ray_start_regular):
|
|||
ray.get(export_definitions_from_worker.remote(f, Actor))
|
||||
|
||||
|
||||
def test_ray_start_and_stop():
|
||||
for i in range(10):
|
||||
subprocess.check_call(["ray", "start", "--head"])
|
||||
subprocess.check_call(["ray", "stop"])
|
||||
|
||||
|
||||
def test_invalid_unicode_in_worker_log(shutdown_only):
|
||||
info = ray.init(num_cpus=1)
|
||||
|
||||
|
|
Loading…
Add table
Reference in a new issue