mirror of
https://github.com/vale981/ray
synced 2025-03-06 18:41:40 -05:00

* Start and clean up workers from the local scheduler Ability to kill workers in photon scheduler Test for old method of starting workers Common codepath for killing workers Common codepath for killing workers Photon test case for starting and killing workers fix build Fix component failure test Register a worker's pid as part of initial connection Address comments and revert photon_connect Set PATH during travis install Fix * Fix photon test case to accept clients on plasma manager fd
121 lines
5.7 KiB
Python
121 lines
5.7 KiB
Python
from __future__ import absolute_import
|
|
from __future__ import division
|
|
from __future__ import print_function
|
|
|
|
import multiprocessing
|
|
import os
|
|
import random
|
|
import subprocess
|
|
import time
|
|
|
|
def random_name():
|
|
return str(random.randint(0, 99999999))
|
|
|
|
def start_local_scheduler(plasma_store_name,
|
|
plasma_manager_name=None,
|
|
worker_path=None,
|
|
plasma_address=None,
|
|
node_ip_address="127.0.0.1",
|
|
redis_address=None,
|
|
use_valgrind=False,
|
|
use_profiler=False,
|
|
redirect_output=False,
|
|
static_resource_list=None,
|
|
num_workers=0):
|
|
"""Start a local scheduler process.
|
|
|
|
Args:
|
|
plasma_store_name (str): The name of the plasma store socket to connect to.
|
|
plasma_manager_name (str): The name of the plasma manager to connect to.
|
|
This does not need to be provided, but if it is, then the Redis address
|
|
must be provided as well.
|
|
worker_path (str): The path of the worker script to use when the local
|
|
scheduler starts up new workers.
|
|
plasma_address (str): The address of the plasma manager to connect to. This
|
|
is only used by the global scheduler to figure out which plasma managers
|
|
are connected to which local schedulers.
|
|
node_ip_address (str): The address of the node that this local scheduler is
|
|
running on.
|
|
redis_address (str): The address of the Redis instance to connect to. If
|
|
this is not provided, then the local scheduler will not connect to Redis.
|
|
use_valgrind (bool): True if the local scheduler should be started inside of
|
|
valgrind. If this is True, use_profiler must be False.
|
|
use_profiler (bool): True if the local scheduler should be started inside a
|
|
profiler. If this is True, use_valgrind must be False.
|
|
redirect_output (bool): True if stdout and stderr should be redirected to
|
|
/dev/null.
|
|
static_resource_list (list): A list of integers specifying the local
|
|
scheduler's resource capacities. The resources should appear in an order
|
|
matching the order defined in task.h.
|
|
num_workers (int): The number of workers that the local scheduler should
|
|
start.
|
|
|
|
Return:
|
|
A tuple of the name of the local scheduler socket and the process ID of the
|
|
local scheduler process.
|
|
"""
|
|
if (plasma_manager_name == None) != (redis_address == None):
|
|
raise Exception("If one of the plasma_manager_name and the redis_address is provided, then both must be provided.")
|
|
if use_valgrind and use_profiler:
|
|
raise Exception("Cannot use valgrind and profiler at the same time.")
|
|
local_scheduler_executable = os.path.join(os.path.dirname(os.path.abspath(__file__)), "../core/src/photon/photon_scheduler")
|
|
local_scheduler_name = "/tmp/scheduler{}".format(random_name())
|
|
command = [local_scheduler_executable,
|
|
"-s", local_scheduler_name,
|
|
"-p", plasma_store_name,
|
|
"-h", node_ip_address,
|
|
"-n", str(num_workers),
|
|
]
|
|
if plasma_manager_name is not None:
|
|
command += ["-m", plasma_manager_name]
|
|
if worker_path is not None:
|
|
assert plasma_store_name is not None
|
|
assert plasma_manager_name is not None
|
|
assert redis_address is not None
|
|
start_worker_command = ("python {} "
|
|
"--node-ip-address={} "
|
|
"--object-store-name={} "
|
|
"--object-store-manager-name={} "
|
|
"--local-scheduler-name={} "
|
|
"--redis-address={}").format(worker_path,
|
|
node_ip_address,
|
|
plasma_store_name,
|
|
plasma_manager_name,
|
|
local_scheduler_name,
|
|
redis_address)
|
|
command += ["-w", start_worker_command]
|
|
if redis_address is not None:
|
|
command += ["-r", redis_address]
|
|
if plasma_address is not None:
|
|
command += ["-a", plasma_address]
|
|
# We want to be able to support independently setting capacity for each of the
|
|
# supported resource types. Thus, the list can be None or contain any number
|
|
# of None values.
|
|
if static_resource_list is None:
|
|
static_resource_list = [None, None]
|
|
if static_resource_list[0] is None:
|
|
# By default, use the number of hardware execution threads for the number of
|
|
# cores.
|
|
static_resource_list[0] = multiprocessing.cpu_count()
|
|
if static_resource_list[1] is None:
|
|
# By default, do not configure any GPUs on this node.
|
|
static_resource_list[1] = 0
|
|
# Pass the resource capacity string to the photon scheduler in all cases.
|
|
# Sanity check to make sure all resource capacities in the list are numeric
|
|
# (int or float).
|
|
assert(all([x == int or x == float for x in map(type, static_resource_list)]))
|
|
command += ["-c", ",".join(map(str, static_resource_list))]
|
|
|
|
with open(os.devnull, "w") as FNULL:
|
|
stdout = FNULL if redirect_output else None
|
|
stderr = FNULL if redirect_output else None
|
|
if use_valgrind:
|
|
pid = subprocess.Popen(["valgrind", "--track-origins=yes", "--leak-check=full", "--show-leak-kinds=all", "--error-exitcode=1"] + command, stdout=stdout, stderr=stderr)
|
|
time.sleep(1.0)
|
|
elif use_profiler:
|
|
pid = subprocess.Popen(["valgrind", "--tool=callgrind"] + command, stdout=stdout, stderr=stderr)
|
|
time.sleep(1.0)
|
|
else:
|
|
pid = subprocess.Popen(command, stdout=stdout, stderr=stderr)
|
|
time.sleep(0.1)
|
|
return local_scheduler_name, pid
|