Skip to content

Commit 2b8e648

Browse files
stephanie-wangrobertnishihara
authored andcommitted
Start and clean up workers from the local scheduler. (#250)
* 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
1 parent ec175b7 commit 2b8e648

File tree

12 files changed

+555
-164
lines changed

12 files changed

+555
-164
lines changed

.travis.yml

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -53,6 +53,8 @@ install:
5353
- ./.travis/install-dependencies.sh
5454
- ./.travis/install-ray.sh
5555

56+
- if [[ "$PYTHON" == "3.5" ]]; then export PATH="$HOME/miniconda/bin:$PATH"; fi
57+
5658
- cd python/core
5759
- bash ../../src/common/test/run_tests.sh
5860
- bash ../../src/plasma/test/run_tests.sh

python/photon/photon_services.py

Lines changed: 19 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -11,11 +11,17 @@
1111
def random_name():
1212
return str(random.randint(0, 99999999))
1313

14-
def start_local_scheduler(plasma_store_name, plasma_manager_name=None,
15-
worker_path=None, plasma_address=None,
16-
node_ip_address="127.0.0.1", redis_address=None,
17-
use_valgrind=False, use_profiler=False,
18-
redirect_output=False, static_resource_list=None):
14+
def start_local_scheduler(plasma_store_name,
15+
plasma_manager_name=None,
16+
worker_path=None,
17+
plasma_address=None,
18+
node_ip_address="127.0.0.1",
19+
redis_address=None,
20+
use_valgrind=False,
21+
use_profiler=False,
22+
redirect_output=False,
23+
static_resource_list=None,
24+
num_workers=0):
1925
"""Start a local scheduler process.
2026
2127
Args:
@@ -41,6 +47,8 @@ def start_local_scheduler(plasma_store_name, plasma_manager_name=None,
4147
static_resource_list (list): A list of integers specifying the local
4248
scheduler's resource capacities. The resources should appear in an order
4349
matching the order defined in task.h.
50+
num_workers (int): The number of workers that the local scheduler should
51+
start.
4452
4553
Return:
4654
A tuple of the name of the local scheduler socket and the process ID of the
@@ -52,7 +60,12 @@ def start_local_scheduler(plasma_store_name, plasma_manager_name=None,
5260
raise Exception("Cannot use valgrind and profiler at the same time.")
5361
local_scheduler_executable = os.path.join(os.path.dirname(os.path.abspath(__file__)), "../core/src/photon/photon_scheduler")
5462
local_scheduler_name = "/tmp/scheduler{}".format(random_name())
55-
command = [local_scheduler_executable, "-s", local_scheduler_name, "-p", plasma_store_name, "-h", node_ip_address]
63+
command = [local_scheduler_executable,
64+
"-s", local_scheduler_name,
65+
"-p", plasma_store_name,
66+
"-h", node_ip_address,
67+
"-n", str(num_workers),
68+
]
5669
if plasma_manager_name is not None:
5770
command += ["-m", plasma_manager_name]
5871
if worker_path is not None:

python/ray/services.py

Lines changed: 71 additions & 27 deletions
Original file line numberDiff line numberDiff line change
@@ -13,6 +13,7 @@
1313
import sys
1414
import time
1515
from collections import namedtuple, OrderedDict
16+
import threading
1617

1718
# Ray modules
1819
import photon
@@ -89,18 +90,24 @@ def kill_process(p):
8990
if RUN_PHOTON_PROFILER or RUN_PLASMA_MANAGER_PROFILER or RUN_PLASMA_STORE_PROFILER:
9091
os.kill(p.pid, signal.SIGINT) # Give process signal to write profiler data.
9192
time.sleep(0.1) # Wait for profiling data to be written.
92-
p.kill()
93-
# Sleeping for 0 should yield the core and allow the killed process to process
94-
# its pending signals.
95-
time.sleep(0)
93+
94+
# Allow the process one second to exit gracefully.
95+
p.terminate()
96+
timer = threading.Timer(1, lambda p: p.kill(), [p])
97+
try:
98+
timer.start()
99+
p.wait()
100+
finally:
101+
timer.cancel()
102+
96103
if p.poll() is not None:
97104
return True
98-
p.terminate()
99-
# Sleeping for 0 should yield the core and allow the killed process to process
100-
# its pending signals.
101-
time.sleep(0)
102-
if p.poll is not None:
105+
106+
# If the process did not exit within one second, force kill it.
107+
p.kill()
108+
if p.poll() is not None:
103109
return True
110+
104111
# The process was not killed for some reason.
105112
return False
106113

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

265-
def start_local_scheduler(redis_address, node_ip_address, plasma_store_name,
266-
plasma_manager_name, worker_path, plasma_address=None,
267-
cleanup=True, redirect_output=False,
268-
static_resource_list=None):
272+
def start_local_scheduler(redis_address,
273+
node_ip_address,
274+
plasma_store_name,
275+
plasma_manager_name,
276+
worker_path,
277+
plasma_address=None,
278+
cleanup=True,
279+
redirect_output=False,
280+
static_resource_list=None,
281+
num_workers=0):
269282
"""Start a local scheduler process.
270283
271284
Args:
@@ -284,6 +297,8 @@ def start_local_scheduler(redis_address, node_ip_address, plasma_store_name,
284297
/dev/null.
285298
static_resource_list (list): An ordered list of the configured resource
286299
capacities for this local scheduler.
300+
num_workers (int): The number of workers that the local scheduler should
301+
start.
287302
288303
Return:
289304
The name of the local scheduler socket.
@@ -296,7 +311,8 @@ def start_local_scheduler(redis_address, node_ip_address, plasma_store_name,
296311
plasma_address=plasma_address,
297312
use_profiler=RUN_PHOTON_PROFILER,
298313
redirect_output=redirect_output,
299-
static_resource_list=static_resource_list)
314+
static_resource_list=static_resource_list,
315+
num_workers=num_workers)
300316
if cleanup:
301317
all_processes[PROCESS_TYPE_LOCAL_SCHEDULER].append(p)
302318
return local_scheduler_name
@@ -391,6 +407,7 @@ def start_ray_processes(address_info=None,
391407
redirect_output=False,
392408
include_global_scheduler=False,
393409
include_redis=False,
410+
start_workers_from_local_scheduler=True,
394411
num_cpus=None,
395412
num_gpus=None):
396413
"""Helper method to start Ray processes.
@@ -417,6 +434,9 @@ def start_ray_processes(address_info=None,
417434
start a global scheduler process.
418435
include_redis (bool): If include_redis is True, then start a Redis server
419436
process.
437+
start_workers_from_local_scheduler (bool): If this flag is True, then start
438+
the initial workers from the local scheduler. Else, start them from
439+
Python.
420440
num_cpus: A list of length num_local_schedulers containing the number of
421441
CPUs each local scheduler should be configured with.
422442
num_gpus: A list of length num_local_schedulers containing the number of
@@ -489,12 +509,25 @@ def start_ray_processes(address_info=None,
489509
object_store_addresses.append(object_store_address)
490510
time.sleep(0.1)
491511

512+
# Determine how many workers to start for each local scheduler.
513+
num_workers_per_local_scheduler = [0] * num_local_schedulers
514+
for i in range(num_workers):
515+
num_workers_per_local_scheduler[i % num_local_schedulers] += 1
516+
492517
# Start any local schedulers that do not yet exist.
493518
for i in range(len(local_scheduler_socket_names), num_local_schedulers):
494519
# Connect the local scheduler to the object store at the same index.
495520
object_store_address = object_store_addresses[i]
496521
plasma_address = "{}:{}".format(node_ip_address,
497522
object_store_address.manager_port)
523+
# Determine how many workers this local scheduler should start.
524+
if start_workers_from_local_scheduler:
525+
num_local_scheduler_workers = num_workers_per_local_scheduler[i]
526+
num_workers_per_local_scheduler[i] = 0
527+
else:
528+
# If we're starting the workers from Python, the local scheduler should
529+
# not start any workers.
530+
num_local_scheduler_workers = 0
498531
# Start the local scheduler.
499532
local_scheduler_name = start_local_scheduler(redis_address,
500533
node_ip_address,
@@ -504,7 +537,8 @@ def start_ray_processes(address_info=None,
504537
plasma_address=plasma_address,
505538
cleanup=cleanup,
506539
redirect_output=redirect_output,
507-
static_resource_list=[num_cpus[i], num_gpus[i]])
540+
static_resource_list=[num_cpus[i], num_gpus[i]],
541+
num_workers=num_local_scheduler_workers)
508542
local_scheduler_socket_names.append(local_scheduler_name)
509543
time.sleep(0.1)
510544

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

516-
# Start the workers.
517-
for i in range(num_workers):
518-
object_store_address = object_store_addresses[i % num_local_schedulers]
519-
local_scheduler_name = local_scheduler_socket_names[i % num_local_schedulers]
520-
start_worker(node_ip_address,
521-
object_store_address.name,
522-
object_store_address.manager_name,
523-
local_scheduler_name,
524-
redis_address,
525-
worker_path,
526-
cleanup=cleanup,
527-
redirect_output=redirect_output)
550+
# Start any workers that the local scheduler has not already started.
551+
for i, num_local_scheduler_workers in enumerate(num_workers_per_local_scheduler):
552+
object_store_address = object_store_addresses[i]
553+
local_scheduler_name = local_scheduler_socket_names[i]
554+
for j in range(num_local_scheduler_workers):
555+
start_worker(node_ip_address,
556+
object_store_address.name,
557+
object_store_address.manager_name,
558+
local_scheduler_name,
559+
redis_address,
560+
worker_path,
561+
cleanup=cleanup,
562+
redirect_output=redirect_output)
563+
num_workers_per_local_scheduler[i] -= 1
564+
565+
# Make sure that we've started all the workers.
566+
assert(sum(num_workers_per_local_scheduler) == 0)
528567

529568
# Return the addresses of the relevant processes.
530569
return address_info
@@ -581,6 +620,7 @@ def start_ray_head(address_info=None,
581620
worker_path=None,
582621
cleanup=True,
583622
redirect_output=False,
623+
start_workers_from_local_scheduler=True,
584624
num_cpus=None,
585625
num_gpus=None):
586626
"""Start Ray in local mode.
@@ -603,6 +643,9 @@ def start_ray_head(address_info=None,
603643
method exits.
604644
redirect_output (bool): True if stdout and stderr should be redirected to
605645
/dev/null.
646+
start_workers_from_local_scheduler (bool): If this flag is True, then start
647+
the initial workers from the local scheduler. Else, start them from
648+
Python.
606649
num_cpus (int): number of cpus to configure the local scheduler with.
607650
num_gpus (int): number of gpus to configure the local scheduler with.
608651
@@ -619,5 +662,6 @@ def start_ray_head(address_info=None,
619662
redirect_output=redirect_output,
620663
include_global_scheduler=True,
621664
include_redis=True,
665+
start_workers_from_local_scheduler=start_workers_from_local_scheduler,
622666
num_cpus=num_cpus,
623667
num_gpus=num_gpus)

python/ray/worker.py

Lines changed: 15 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -735,9 +735,15 @@ def get_address_info_from_redis(redis_address, node_ip_address, num_retries=5):
735735
time.sleep(1)
736736
counter += 1
737737

738-
def _init(address_info=None, start_ray_local=False, object_id_seed=None,
739-
num_workers=None, num_local_schedulers=None,
740-
driver_mode=SCRIPT_MODE, num_cpus=None, num_gpus=None):
738+
def _init(address_info=None,
739+
start_ray_local=False,
740+
object_id_seed=None,
741+
num_workers=None,
742+
num_local_schedulers=None,
743+
driver_mode=SCRIPT_MODE,
744+
start_workers_from_local_scheduler=True,
745+
num_cpus=None,
746+
num_gpus=None):
741747
"""Helper method to connect to an existing Ray cluster or start a new one.
742748
743749
This method handles two cases. Either a Ray cluster already exists and we
@@ -764,6 +770,9 @@ def _init(address_info=None, start_ray_local=False, object_id_seed=None,
764770
only provided if start_ray_local is True.
765771
driver_mode (bool): The mode in which to start the driver. This should be
766772
one of ray.SCRIPT_MODE, ray.PYTHON_MODE, and ray.SILENT_MODE.
773+
start_workers_from_local_scheduler (bool): If this flag is True, then start
774+
the initial workers from the local scheduler. Else, start them from
775+
Python. The latter case is for debugging purposes only.
767776
num_cpus: A list containing the number of CPUs the local schedulers should
768777
be configured with.
769778
num_gpus: A list containing the number of GPUs the local schedulers should
@@ -815,7 +824,9 @@ def _init(address_info=None, start_ray_local=False, object_id_seed=None,
815824
node_ip_address=node_ip_address,
816825
num_workers=num_workers,
817826
num_local_schedulers=num_local_schedulers,
818-
num_cpus=num_cpus, num_gpus=num_gpus)
827+
start_workers_from_local_scheduler=start_workers_from_local_scheduler,
828+
num_cpus=num_cpus,
829+
num_gpus=num_gpus)
819830
else:
820831
if redis_address is None:
821832
raise Exception("If start_ray_local=False, then redis_address must be provided.")

src/photon/photon.h

Lines changed: 13 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -27,20 +27,14 @@ enum photon_message_type {
2727
RECONSTRUCT_OBJECT,
2828
/** Log a message to the event table. */
2929
EVENT_LOG_MESSAGE,
30+
/** Register a worker's process ID with the local scheduler. */
31+
REGISTER_PID,
3032
};
3133

32-
// clang-format off
33-
/** Contains all information that is associated to a worker. */
34-
typedef struct {
35-
int sock;
36-
/** A pointer to a task object, to update the task table. */
37-
task *task_in_progress;
38-
} worker;
39-
// clang-format on
40-
4134
/* These are needed to define the UT_arrays. */
4235
UT_icd task_ptr_icd;
43-
UT_icd worker_icd;
36+
UT_icd workers_icd;
37+
UT_icd pid_t_icd;
4438

4539
/** Internal state of the scheduling algorithm. */
4640
typedef struct scheduling_algorithm_state scheduling_algorithm_state;
@@ -50,7 +44,7 @@ typedef struct scheduling_algorithm_state scheduling_algorithm_state;
5044
* scheduler. */
5145
typedef struct {
5246
/** The script to use when starting a new worker. */
53-
char *start_worker_command;
47+
const char **start_worker_command;
5448
/** Whether there is a global scheduler. */
5549
bool global_scheduler_exists;
5650
} local_scheduler_config;
@@ -65,6 +59,9 @@ typedef struct {
6559
* structs when we free the scheduler state and also to access the worker
6660
* structs in the tests. */
6761
UT_array *workers;
62+
/** List of the process IDs for child processes (workers) started by the
63+
* local scheduler that have not sent a REGISTER_PID message yet. */
64+
UT_array *child_pids;
6865
/** The handle to the database. */
6966
db_handle *db;
7067
/** The Plasma client. */
@@ -90,6 +87,11 @@ typedef struct {
9087
* no task is running on the worker, this will be NULL. This is used to
9188
* update the task table. */
9289
task *task_in_progress;
90+
/** The process ID of the client. If this is set to zero, the client has not
91+
* yet registered a process ID. */
92+
pid_t pid;
93+
/** Whether the client is a child process of the local scheduler. */
94+
bool is_child;
9395
/** A pointer to the local scheduler state. */
9496
local_scheduler_state *local_scheduler_state;
9597
} local_scheduler_client;

src/photon/photon_client.c

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -7,6 +7,11 @@
77
photon_conn *photon_connect(const char *photon_socket) {
88
photon_conn *result = malloc(sizeof(photon_conn));
99
result->conn = connect_ipc_sock(photon_socket);
10+
/* If this is a worker, register the process ID with the local scheduler. */
11+
pid_t my_pid = getpid();
12+
int success = write_message(result->conn, REGISTER_PID, sizeof(my_pid),
13+
(uint8_t *) &my_pid);
14+
CHECKM(success == 0, "Unable to register worker with local scheduler");
1015
return result;
1116
}
1217

src/photon/photon_extension.c

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@ static int PyPhotonClient_init(PyPhotonClient *self,
2020
if (!PyArg_ParseTuple(args, "s", &socket_name)) {
2121
return -1;
2222
}
23+
/* Connect to the Photon scheduler. */
2324
self->photon_connection = photon_connect(socket_name);
2425
return 0;
2526
}

0 commit comments

Comments
 (0)