From 898595adf975ae9a05fc36a581c8d923e1056333 Mon Sep 17 00:00:00 2001 From: Robert Nishihara Date: Sun, 6 Nov 2016 17:57:28 -0800 Subject: [PATCH] Export cached reusable variables before cached remote functions. --- lib/python/ray/worker.py | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/lib/python/ray/worker.py b/lib/python/ray/worker.py index 22ed54f1bb5c..40c7c6b968a4 100644 --- a/lib/python/ray/worker.py +++ b/lib/python/ray/worker.py @@ -873,15 +873,25 @@ def connect(address_info, mode=WORKER_MODE, worker=global_worker): current_directory = os.path.abspath(os.path.curdir) worker.run_function_on_all_workers(lambda worker: sys.path.insert(1, script_directory)) worker.run_function_on_all_workers(lambda worker: sys.path.insert(1, current_directory)) + # TODO(rkn): Here we first export functions to run, then reusable variables, + # then remote functions. The order matters. For example, one of the + # functions to run may set the Python path, which is needed to import a + # module used to define a reusable variable, which in turn is used inside a + # remote function. We may want to change the order to simply be the order in + # which the exports were defined on the driver. In addition, we will need to + # retain the ability to decide what the first few exports are (mostly to set + # the Python path). Additionally, note that the first exports to be defined + # on the driver will be the ones defined in separate modules that are + # imported by the driver. # Export cached functions_to_run. for function in worker.cached_functions_to_run: worker.run_function_on_all_workers(function) - # Export cached remote functions to the workers. - for function_id, func_name, func, num_return_vals in worker.cached_remote_functions: - export_remote_function(function_id, func_name, func, num_return_vals, worker) # Export cached reusable variables to the workers. for name, reusable_variable in reusables._cached_reusables: reusables.__setattr__(name, reusable_variable) + # Export cached remote functions to the workers. + for function_id, func_name, func, num_return_vals in worker.cached_remote_functions: + export_remote_function(function_id, func_name, func, num_return_vals, worker) worker.cached_functions_to_run = None worker.cached_remote_functions = None reusables._cached_reusables = None