Skip to content

Commit

Permalink
Start and clean up workers from the local scheduler. (#250)
Browse files Browse the repository at this point in the history
* 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
  • Loading branch information
stephanie-wang authored and robertnishihara committed Feb 10, 2017
1 parent ec175b7 commit 2b8e648
Show file tree
Hide file tree
Showing 12 changed files with 555 additions and 164 deletions.
2 changes: 2 additions & 0 deletions .travis.yml
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,8 @@ install:
- ./.travis/install-dependencies.sh
- ./.travis/install-ray.sh

- if [[ "$PYTHON" == "3.5" ]]; then export PATH="$HOME/miniconda/bin:$PATH"; fi

- cd python/core
- bash ../../src/common/test/run_tests.sh
- bash ../../src/plasma/test/run_tests.sh
Expand Down
25 changes: 19 additions & 6 deletions python/photon/photon_services.py
Original file line number Diff line number Diff line change
Expand Up @@ -11,11 +11,17 @@
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):
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:
Expand All @@ -41,6 +47,8 @@ def start_local_scheduler(plasma_store_name, plasma_manager_name=None,
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
Expand All @@ -52,7 +60,12 @@ def start_local_scheduler(plasma_store_name, plasma_manager_name=None,
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]
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:
Expand Down
98 changes: 71 additions & 27 deletions python/ray/services.py
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@
import sys
import time
from collections import namedtuple, OrderedDict
import threading

# Ray modules
import photon
Expand Down Expand Up @@ -89,18 +90,24 @@ def kill_process(p):
if RUN_PHOTON_PROFILER or RUN_PLASMA_MANAGER_PROFILER or RUN_PLASMA_STORE_PROFILER:
os.kill(p.pid, signal.SIGINT) # Give process signal to write profiler data.
time.sleep(0.1) # Wait for profiling data to be written.
p.kill()
# Sleeping for 0 should yield the core and allow the killed process to process
# its pending signals.
time.sleep(0)

# Allow the process one second to exit gracefully.
p.terminate()
timer = threading.Timer(1, lambda p: p.kill(), [p])
try:
timer.start()
p.wait()
finally:
timer.cancel()

if p.poll() is not None:
return True
p.terminate()
# Sleeping for 0 should yield the core and allow the killed process to process
# its pending signals.
time.sleep(0)
if p.poll is not None:

# If the process did not exit within one second, force kill it.
p.kill()
if p.poll() is not None:
return True

# The process was not killed for some reason.
return False

Expand Down Expand Up @@ -262,10 +269,16 @@ def start_global_scheduler(redis_address, cleanup=True, redirect_output=False):
if cleanup:
all_processes[PROCESS_TYPE_GLOBAL_SCHEDULER].append(p)

def start_local_scheduler(redis_address, node_ip_address, plasma_store_name,
plasma_manager_name, worker_path, plasma_address=None,
cleanup=True, redirect_output=False,
static_resource_list=None):
def start_local_scheduler(redis_address,
node_ip_address,
plasma_store_name,
plasma_manager_name,
worker_path,
plasma_address=None,
cleanup=True,
redirect_output=False,
static_resource_list=None,
num_workers=0):
"""Start a local scheduler process.
Args:
Expand All @@ -284,6 +297,8 @@ def start_local_scheduler(redis_address, node_ip_address, plasma_store_name,
/dev/null.
static_resource_list (list): An ordered list of the configured resource
capacities for this local scheduler.
num_workers (int): The number of workers that the local scheduler should
start.
Return:
The name of the local scheduler socket.
Expand All @@ -296,7 +311,8 @@ def start_local_scheduler(redis_address, node_ip_address, plasma_store_name,
plasma_address=plasma_address,
use_profiler=RUN_PHOTON_PROFILER,
redirect_output=redirect_output,
static_resource_list=static_resource_list)
static_resource_list=static_resource_list,
num_workers=num_workers)
if cleanup:
all_processes[PROCESS_TYPE_LOCAL_SCHEDULER].append(p)
return local_scheduler_name
Expand Down Expand Up @@ -391,6 +407,7 @@ def start_ray_processes(address_info=None,
redirect_output=False,
include_global_scheduler=False,
include_redis=False,
start_workers_from_local_scheduler=True,
num_cpus=None,
num_gpus=None):
"""Helper method to start Ray processes.
Expand All @@ -417,6 +434,9 @@ def start_ray_processes(address_info=None,
start a global scheduler process.
include_redis (bool): If include_redis is True, then start a Redis server
process.
start_workers_from_local_scheduler (bool): If this flag is True, then start
the initial workers from the local scheduler. Else, start them from
Python.
num_cpus: A list of length num_local_schedulers containing the number of
CPUs each local scheduler should be configured with.
num_gpus: A list of length num_local_schedulers containing the number of
Expand Down Expand Up @@ -489,12 +509,25 @@ def start_ray_processes(address_info=None,
object_store_addresses.append(object_store_address)
time.sleep(0.1)

# Determine how many workers to start for each local scheduler.
num_workers_per_local_scheduler = [0] * num_local_schedulers
for i in range(num_workers):
num_workers_per_local_scheduler[i % num_local_schedulers] += 1

# Start any local schedulers that do not yet exist.
for i in range(len(local_scheduler_socket_names), num_local_schedulers):
# Connect the local scheduler to the object store at the same index.
object_store_address = object_store_addresses[i]
plasma_address = "{}:{}".format(node_ip_address,
object_store_address.manager_port)
# Determine how many workers this local scheduler should start.
if start_workers_from_local_scheduler:
num_local_scheduler_workers = num_workers_per_local_scheduler[i]
num_workers_per_local_scheduler[i] = 0
else:
# If we're starting the workers from Python, the local scheduler should
# not start any workers.
num_local_scheduler_workers = 0
# Start the local scheduler.
local_scheduler_name = start_local_scheduler(redis_address,
node_ip_address,
Expand All @@ -504,7 +537,8 @@ def start_ray_processes(address_info=None,
plasma_address=plasma_address,
cleanup=cleanup,
redirect_output=redirect_output,
static_resource_list=[num_cpus[i], num_gpus[i]])
static_resource_list=[num_cpus[i], num_gpus[i]],
num_workers=num_local_scheduler_workers)
local_scheduler_socket_names.append(local_scheduler_name)
time.sleep(0.1)

Expand All @@ -513,18 +547,23 @@ def start_ray_processes(address_info=None,
assert len(object_store_addresses) == num_local_schedulers
assert len(local_scheduler_socket_names) == num_local_schedulers

# Start the workers.
for i in range(num_workers):
object_store_address = object_store_addresses[i % num_local_schedulers]
local_scheduler_name = local_scheduler_socket_names[i % num_local_schedulers]
start_worker(node_ip_address,
object_store_address.name,
object_store_address.manager_name,
local_scheduler_name,
redis_address,
worker_path,
cleanup=cleanup,
redirect_output=redirect_output)
# Start any workers that the local scheduler has not already started.
for i, num_local_scheduler_workers in enumerate(num_workers_per_local_scheduler):
object_store_address = object_store_addresses[i]
local_scheduler_name = local_scheduler_socket_names[i]
for j in range(num_local_scheduler_workers):
start_worker(node_ip_address,
object_store_address.name,
object_store_address.manager_name,
local_scheduler_name,
redis_address,
worker_path,
cleanup=cleanup,
redirect_output=redirect_output)
num_workers_per_local_scheduler[i] -= 1

# Make sure that we've started all the workers.
assert(sum(num_workers_per_local_scheduler) == 0)

# Return the addresses of the relevant processes.
return address_info
Expand Down Expand Up @@ -581,6 +620,7 @@ def start_ray_head(address_info=None,
worker_path=None,
cleanup=True,
redirect_output=False,
start_workers_from_local_scheduler=True,
num_cpus=None,
num_gpus=None):
"""Start Ray in local mode.
Expand All @@ -603,6 +643,9 @@ def start_ray_head(address_info=None,
method exits.
redirect_output (bool): True if stdout and stderr should be redirected to
/dev/null.
start_workers_from_local_scheduler (bool): If this flag is True, then start
the initial workers from the local scheduler. Else, start them from
Python.
num_cpus (int): number of cpus to configure the local scheduler with.
num_gpus (int): number of gpus to configure the local scheduler with.
Expand All @@ -619,5 +662,6 @@ def start_ray_head(address_info=None,
redirect_output=redirect_output,
include_global_scheduler=True,
include_redis=True,
start_workers_from_local_scheduler=start_workers_from_local_scheduler,
num_cpus=num_cpus,
num_gpus=num_gpus)
19 changes: 15 additions & 4 deletions python/ray/worker.py
Original file line number Diff line number Diff line change
Expand Up @@ -735,9 +735,15 @@ def get_address_info_from_redis(redis_address, node_ip_address, num_retries=5):
time.sleep(1)
counter += 1

def _init(address_info=None, start_ray_local=False, object_id_seed=None,
num_workers=None, num_local_schedulers=None,
driver_mode=SCRIPT_MODE, num_cpus=None, num_gpus=None):
def _init(address_info=None,
start_ray_local=False,
object_id_seed=None,
num_workers=None,
num_local_schedulers=None,
driver_mode=SCRIPT_MODE,
start_workers_from_local_scheduler=True,
num_cpus=None,
num_gpus=None):
"""Helper method to connect to an existing Ray cluster or start a new one.
This method handles two cases. Either a Ray cluster already exists and we
Expand All @@ -764,6 +770,9 @@ def _init(address_info=None, start_ray_local=False, object_id_seed=None,
only provided if start_ray_local is True.
driver_mode (bool): The mode in which to start the driver. This should be
one of ray.SCRIPT_MODE, ray.PYTHON_MODE, and ray.SILENT_MODE.
start_workers_from_local_scheduler (bool): If this flag is True, then start
the initial workers from the local scheduler. Else, start them from
Python. The latter case is for debugging purposes only.
num_cpus: A list containing the number of CPUs the local schedulers should
be configured with.
num_gpus: A list containing the number of GPUs the local schedulers should
Expand Down Expand Up @@ -815,7 +824,9 @@ def _init(address_info=None, start_ray_local=False, object_id_seed=None,
node_ip_address=node_ip_address,
num_workers=num_workers,
num_local_schedulers=num_local_schedulers,
num_cpus=num_cpus, num_gpus=num_gpus)
start_workers_from_local_scheduler=start_workers_from_local_scheduler,
num_cpus=num_cpus,
num_gpus=num_gpus)
else:
if redis_address is None:
raise Exception("If start_ray_local=False, then redis_address must be provided.")
Expand Down
24 changes: 13 additions & 11 deletions src/photon/photon.h
Original file line number Diff line number Diff line change
Expand Up @@ -27,20 +27,14 @@ enum photon_message_type {
RECONSTRUCT_OBJECT,
/** Log a message to the event table. */
EVENT_LOG_MESSAGE,
/** Register a worker's process ID with the local scheduler. */
REGISTER_PID,
};

// clang-format off
/** Contains all information that is associated to a worker. */
typedef struct {
int sock;
/** A pointer to a task object, to update the task table. */
task *task_in_progress;
} worker;
// clang-format on

/* These are needed to define the UT_arrays. */
UT_icd task_ptr_icd;
UT_icd worker_icd;
UT_icd workers_icd;
UT_icd pid_t_icd;

/** Internal state of the scheduling algorithm. */
typedef struct scheduling_algorithm_state scheduling_algorithm_state;
Expand All @@ -50,7 +44,7 @@ typedef struct scheduling_algorithm_state scheduling_algorithm_state;
* scheduler. */
typedef struct {
/** The script to use when starting a new worker. */
char *start_worker_command;
const char **start_worker_command;
/** Whether there is a global scheduler. */
bool global_scheduler_exists;
} local_scheduler_config;
Expand All @@ -65,6 +59,9 @@ typedef struct {
* structs when we free the scheduler state and also to access the worker
* structs in the tests. */
UT_array *workers;
/** List of the process IDs for child processes (workers) started by the
* local scheduler that have not sent a REGISTER_PID message yet. */
UT_array *child_pids;
/** The handle to the database. */
db_handle *db;
/** The Plasma client. */
Expand All @@ -90,6 +87,11 @@ typedef struct {
* no task is running on the worker, this will be NULL. This is used to
* update the task table. */
task *task_in_progress;
/** The process ID of the client. If this is set to zero, the client has not
* yet registered a process ID. */
pid_t pid;
/** Whether the client is a child process of the local scheduler. */
bool is_child;
/** A pointer to the local scheduler state. */
local_scheduler_state *local_scheduler_state;
} local_scheduler_client;
Expand Down
5 changes: 5 additions & 0 deletions src/photon/photon_client.c
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,11 @@
photon_conn *photon_connect(const char *photon_socket) {
photon_conn *result = malloc(sizeof(photon_conn));
result->conn = connect_ipc_sock(photon_socket);
/* If this is a worker, register the process ID with the local scheduler. */
pid_t my_pid = getpid();
int success = write_message(result->conn, REGISTER_PID, sizeof(my_pid),
(uint8_t *) &my_pid);
CHECKM(success == 0, "Unable to register worker with local scheduler");
return result;
}

Expand Down
1 change: 1 addition & 0 deletions src/photon/photon_extension.c
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ static int PyPhotonClient_init(PyPhotonClient *self,
if (!PyArg_ParseTuple(args, "s", &socket_name)) {
return -1;
}
/* Connect to the Photon scheduler. */
self->photon_connection = photon_connect(socket_name);
return 0;
}
Expand Down
Loading

0 comments on commit 2b8e648

Please sign in to comment.