Skip to content

Commit

Permalink
Export cached reusable variables before cached remote functions.
Browse files Browse the repository at this point in the history
  • Loading branch information
robertnishihara committed Nov 7, 2016
1 parent ca3a5ad commit 898595a
Showing 1 changed file with 13 additions and 3 deletions.
16 changes: 13 additions & 3 deletions lib/python/ray/worker.py
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down

0 comments on commit 898595a

Please sign in to comment.