13
13
import sys
14
14
import time
15
15
from collections import namedtuple , OrderedDict
16
+ import threading
16
17
17
18
# Ray modules
18
19
import photon
@@ -89,18 +90,24 @@ def kill_process(p):
89
90
if RUN_PHOTON_PROFILER or RUN_PLASMA_MANAGER_PROFILER or RUN_PLASMA_STORE_PROFILER :
90
91
os .kill (p .pid , signal .SIGINT ) # Give process signal to write profiler data.
91
92
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
+
96
103
if p .poll () is not None :
97
104
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 :
103
109
return True
110
+
104
111
# The process was not killed for some reason.
105
112
return False
106
113
@@ -262,10 +269,16 @@ def start_global_scheduler(redis_address, cleanup=True, redirect_output=False):
262
269
if cleanup :
263
270
all_processes [PROCESS_TYPE_GLOBAL_SCHEDULER ].append (p )
264
271
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 ):
269
282
"""Start a local scheduler process.
270
283
271
284
Args:
@@ -284,6 +297,8 @@ def start_local_scheduler(redis_address, node_ip_address, plasma_store_name,
284
297
/dev/null.
285
298
static_resource_list (list): An ordered list of the configured resource
286
299
capacities for this local scheduler.
300
+ num_workers (int): The number of workers that the local scheduler should
301
+ start.
287
302
288
303
Return:
289
304
The name of the local scheduler socket.
@@ -296,7 +311,8 @@ def start_local_scheduler(redis_address, node_ip_address, plasma_store_name,
296
311
plasma_address = plasma_address ,
297
312
use_profiler = RUN_PHOTON_PROFILER ,
298
313
redirect_output = redirect_output ,
299
- static_resource_list = static_resource_list )
314
+ static_resource_list = static_resource_list ,
315
+ num_workers = num_workers )
300
316
if cleanup :
301
317
all_processes [PROCESS_TYPE_LOCAL_SCHEDULER ].append (p )
302
318
return local_scheduler_name
@@ -391,6 +407,7 @@ def start_ray_processes(address_info=None,
391
407
redirect_output = False ,
392
408
include_global_scheduler = False ,
393
409
include_redis = False ,
410
+ start_workers_from_local_scheduler = True ,
394
411
num_cpus = None ,
395
412
num_gpus = None ):
396
413
"""Helper method to start Ray processes.
@@ -417,6 +434,9 @@ def start_ray_processes(address_info=None,
417
434
start a global scheduler process.
418
435
include_redis (bool): If include_redis is True, then start a Redis server
419
436
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.
420
440
num_cpus: A list of length num_local_schedulers containing the number of
421
441
CPUs each local scheduler should be configured with.
422
442
num_gpus: A list of length num_local_schedulers containing the number of
@@ -489,12 +509,25 @@ def start_ray_processes(address_info=None,
489
509
object_store_addresses .append (object_store_address )
490
510
time .sleep (0.1 )
491
511
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
+
492
517
# Start any local schedulers that do not yet exist.
493
518
for i in range (len (local_scheduler_socket_names ), num_local_schedulers ):
494
519
# Connect the local scheduler to the object store at the same index.
495
520
object_store_address = object_store_addresses [i ]
496
521
plasma_address = "{}:{}" .format (node_ip_address ,
497
522
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
498
531
# Start the local scheduler.
499
532
local_scheduler_name = start_local_scheduler (redis_address ,
500
533
node_ip_address ,
@@ -504,7 +537,8 @@ def start_ray_processes(address_info=None,
504
537
plasma_address = plasma_address ,
505
538
cleanup = cleanup ,
506
539
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 )
508
542
local_scheduler_socket_names .append (local_scheduler_name )
509
543
time .sleep (0.1 )
510
544
@@ -513,18 +547,23 @@ def start_ray_processes(address_info=None,
513
547
assert len (object_store_addresses ) == num_local_schedulers
514
548
assert len (local_scheduler_socket_names ) == num_local_schedulers
515
549
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 )
528
567
529
568
# Return the addresses of the relevant processes.
530
569
return address_info
@@ -581,6 +620,7 @@ def start_ray_head(address_info=None,
581
620
worker_path = None ,
582
621
cleanup = True ,
583
622
redirect_output = False ,
623
+ start_workers_from_local_scheduler = True ,
584
624
num_cpus = None ,
585
625
num_gpus = None ):
586
626
"""Start Ray in local mode.
@@ -603,6 +643,9 @@ def start_ray_head(address_info=None,
603
643
method exits.
604
644
redirect_output (bool): True if stdout and stderr should be redirected to
605
645
/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.
606
649
num_cpus (int): number of cpus to configure the local scheduler with.
607
650
num_gpus (int): number of gpus to configure the local scheduler with.
608
651
@@ -619,5 +662,6 @@ def start_ray_head(address_info=None,
619
662
redirect_output = redirect_output ,
620
663
include_global_scheduler = True ,
621
664
include_redis = True ,
665
+ start_workers_from_local_scheduler = start_workers_from_local_scheduler ,
622
666
num_cpus = num_cpus ,
623
667
num_gpus = num_gpus )
0 commit comments