diff --git a/python/ray/tests/test_ray_init.py b/python/ray/tests/test_ray_init.py index 95bdd5c79..b28b521fe 100644 --- a/python/ray/tests/test_ray_init.py +++ b/python/ray/tests/test_ray_init.py @@ -75,6 +75,20 @@ class TestRedisPassword: ray.get(object_ref) +def test_shutdown_and_reset_global_worker(shutdown_only): + ray.init(job_config=ray.job_config.JobConfig(code_search_path=["a"])) + ray.shutdown() + ray.init() + + @ray.remote + class A: + def f(self): + return 100 + + a = A.remote() + ray.get(a.f.remote()) + + if __name__ == "__main__": import pytest import sys diff --git a/python/ray/worker.py b/python/ray/worker.py index 7239b80a9..b57febc84 100644 --- a/python/ray/worker.py +++ b/python/ray/worker.py @@ -772,6 +772,17 @@ def init( job_config=job_config) if job_config and job_config.code_search_path: global_worker.set_load_code_from_local(True) + else: + # Because `ray.shutdown()` doesn't reset this flag, for multiple + # sessions in one process, the 2nd `ray.init()` will reuse the + # flag of last session. For example: + # ray.init(load_code_from_local=True) + # ray.shutdown() + # ray.init() + # # Here the flag `load_code_from_local` is still True if we + # # doesn't have this `else` branch. + # ray.shutdown() + global_worker.set_load_code_from_local(False) for hook in _post_init_hooks: hook()