From e557b6a9a0e0878c47d3b103493b3b7960dac995 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Sun, 4 Nov 2018 21:44:53 -0800 Subject: [PATCH 01/74] [tune] Throw on overstepping --- python/ray/tune/test/trial_runner_test.py | 68 ++++++++++++++++++++--- python/ray/tune/trial_runner.py | 14 ++--- 2 files changed, 68 insertions(+), 14 deletions(-) diff --git a/python/ray/tune/test/trial_runner_test.py b/python/ray/tune/test/trial_runner_test.py index 3c9ae43e6a78..03430d3d0c93 100644 --- a/python/ray/tune/test/trial_runner_test.py +++ b/python/ray/tune/test/trial_runner_test.py @@ -967,6 +967,31 @@ def testMultiStepRun(self): self.assertEqual(trials[0].status, Trial.RUNNING) self.assertEqual(trials[1].status, Trial.RUNNING) + def testMultiStepRun2(self): + """Checks that runner.step throws when overstepping.""" + ray.init(num_cpus=1) + runner = TrialRunner(BasicVariantGenerator()) + kwargs = { + "stopping_criterion": { + "training_iteration": 2 + }, + "resources": Resources(cpu=1), + } + trials = [Trial("__fake", **kwargs), Trial("__fake", **kwargs)] + for t in trials: + runner.add_trial(t) + + runner.step() + self.assertEqual(trials[0].status, Trial.RUNNING) + + runner.step() + self.assertEqual(trials[0].status, Trial.RUNNING) + + runner.step() + self.assertEqual(trials[0].status, Trial.TERMINATED) + self.assertRaises(TuneError, runner.step) + + def testErrorHandling(self): ray.init(num_cpus=4, num_gpus=2) runner = TrialRunner(BasicVariantGenerator()) @@ -989,6 +1014,12 @@ def testErrorHandling(self): self.assertEqual(trials[0].status, Trial.ERROR) self.assertEqual(trials[1].status, Trial.RUNNING) + def testThrowOnOverstep(self): + ray.init(num_cpus=1, num_gpus=1) + runner = TrialRunner(BasicVariantGenerator()) + runner.step() + self.assertRaises(TuneError, runner.step) + def testFailureRecoveryDisabled(self): ray.init(num_cpus=1, num_gpus=1) runner = TrialRunner(BasicVariantGenerator()) @@ -1387,17 +1418,29 @@ def testSearchAlgStalled(self): self.assertTrue(runner.is_finished()) def testSearchAlgFinishes(self): - """SearchAlg changing state in `next_trials` does not crash.""" + """Empty SearchAlg changing state in `next_trials` does not crash.""" class FinishFastAlg(SuggestionAlgorithm): + _index = 0 def next_trials(self): - self._finished = True - return [] + trials = [] + self._index += 1 - ray.init(num_cpus=4, num_gpus=2) + for trial in self._trial_generator: + trials += [trial] + break + + if self._index > 4: + self._finished = True + return trials + + def _suggest(self, trial_id): + return {} + + ray.init(num_cpus=2) experiment_spec = { "run": "__fake", - "num_samples": 3, + "num_samples": 2, "stop": { "training_iteration": 1 } @@ -1407,9 +1450,20 @@ def next_trials(self): searcher.add_configurations(experiments) runner = TrialRunner(search_alg=searcher) - runner.step() # This should not fail + self.assertFalse(runner.is_finished()) + runner.step() # This launches a new run + runner.step() # This launches a 2nd run + self.assertFalse(searcher.is_finished()) + self.assertFalse(runner.is_finished()) + runner.step() # This kills the first run + self.assertFalse(searcher.is_finished()) + self.assertFalse(runner.is_finished()) + runner.step() # This kills the 2nd run + self.assertFalse(searcher.is_finished()) + self.assertFalse(runner.is_finished()) + runner.step() # this converts self._finished to True self.assertTrue(searcher.is_finished()) - self.assertTrue(runner.is_finished()) + self.assertRaises(TuneError, runner.step) if __name__ == "__main__": diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index 8e3eb861246f..2ee831367112 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -108,16 +108,14 @@ def step(self): Callers should typically run this method repeatedly in a loop. They may inspect or modify the runner's state in between calls to step(). """ + if self.is_finished(): + raise TuneError("Called step when all trials finished?") self.trial_executor.on_step_begin() next_trial = self._get_next_trial() if next_trial is not None: self.trial_executor.start_trial(next_trial) elif self.trial_executor.get_running_trials(): self._process_events() - elif self.is_finished(): - # We check `is_finished` again here because the experiment - # may have finished while getting the next trial. - pass else: for trial in self._trials: if trial.status == Trial.PENDING: @@ -137,7 +135,7 @@ def step(self): raise TuneError( "There are paused trials, but no more pending " "trials with sufficient resources.") - raise TuneError("Called step when all trials finished?") + if self._server: self._process_requests() @@ -306,13 +304,15 @@ def _update_trial_queue(self, blocking=False, timeout=600): Args: blocking (bool): Blocks until either a trial is available - or the Runner finishes (i.e., timeout or search algorithm - finishes). + or is_finished (timeout or search algorithm finishes). timeout (int): Seconds before blocking times out. """ trials = self._search_alg.next_trials() if blocking and not trials: start = time.time() + # Checking `is_finished` instead of _search_alg.is_finished + # is fine because blocking only occurs if all trials are + # finished and search_algorithm is not yet finished while (not trials and not self.is_finished() and time.time() - start < timeout): logger.info("Blocking for next trial...") From b75578586a8cbdea42b79a528eda1071e2dc228d Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Sun, 4 Nov 2018 23:28:09 -0800 Subject: [PATCH 02/74] Add Tune Multi-Node Tests --- python/ray/test/cluster_utils.py | 8 +- python/ray/tune/test/cluster_tests.py | 151 +++++++++++++++++++++ test/jenkins_tests/run_multi_node_tests.sh | 3 + 3 files changed, 160 insertions(+), 2 deletions(-) create mode 100644 python/ray/tune/test/cluster_tests.py diff --git a/python/ray/test/cluster_utils.py b/python/ray/test/cluster_utils.py index c4cf2b801623..fa55f9e1b163 100644 --- a/python/ray/test/cluster_utils.py +++ b/python/ray/test/cluster_utils.py @@ -37,7 +37,8 @@ def __init__(self, head_node_args = head_node_args or {} self.add_node(**head_node_args) if connect: - ray.init(redis_address=self.redis_address) + output_info = ray.init(redis_address=self.redis_address) + logger.info(output_info) def add_node(self, **override_kwargs): """Adds a node to the local Ray Cluster. @@ -153,7 +154,10 @@ def shutdown(self): all_nodes = list(self.worker_nodes) for node in all_nodes: self.remove_node(node) - self.remove_node(self.head_node) + if self.head_node: + self.remove_node(self.head_node) + else: + logger.warning("No headnode exists!") class Node(object): diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py new file mode 100644 index 000000000000..57268b28ee69 --- /dev/null +++ b/python/ray/tune/test/cluster_tests.py @@ -0,0 +1,151 @@ +from __future__ import absolute_import +from __future__ import division +from __future__ import print_function + +import pytest + +from ray.test.cluster_utils import Cluster +import ray +from ray import tune +from ray.tune.error import TuneError +from ray.tune.experiment import Experiment +from ray.tune.trial import Trial, Resources +from ray.tune.trial_runner import TrialRunner +from ray.tune.suggest import grid_search, BasicVariantGenerator + + +@pytest.fixture +def start_connected_cluster(): + # Start the Ray processes. + cluster = Cluster( + initialize_head=True, connect=True, + head_node_args={"resources": dict(CPU=1)}) + yield cluster + # The code after the yield will run as teardown code. + ray.shutdown() + cluster.shutdown() + + +class _Train(tune.Trainable): + def _setup(self, config): + self.state = {"hi": 1} + + def _train(self): + self.state["hi"] += 1 + return {} + + def _save(self, path): + return self.state + + def _restore(self, state): + self.state = state + + +def test_counting_resources(start_connected_cluster): + """Removing a node should cause a Trial to be requeued.""" + cluster = start_connected_cluster + node = cluster.add_node(resources=dict(CPU=1)) + + runner = TrialRunner(BasicVariantGenerator()) + kwargs = { + "stopping_criterion": { + "training_iteration": 3 + } + } + + tune.register_trainable("test", _Train) + trials = [Trial("test", **kwargs), Trial("test", **kwargs)] + for t in trials: + runner.add_trial(t) + + runner.step() # run 1 + cluster.remove_node(node) + runner.step() # run 2 + node = cluster.add_node(resources=dict(CPU=1)) + + assert all(t.status == Trial.RUNNING for t in trials) + + runner.step() # 1 result + print(runner.debug_string()) + + cluster.remove_node(node) + + runner.step() # recover + for i in range(5): + runner.step() + assert all(t.status == Trial.TERMINATED for t in trials) + + with pytest.raises(TuneError): + runner.step() + + +def test_remove_node_before_result(start_connected_cluster): + """Removing a node should cause a Trial to be requeued.""" + cluster = start_connected_cluster + node = cluster.add_node(resources=dict(CPU=1)) + + runner = TrialRunner(BasicVariantGenerator()) + kwargs = { + "stopping_criterion": { + "training_iteration": 3 + } + } + + tune.register_trainable("test", _Train) + trials = [Trial("test", **kwargs), Trial("test", **kwargs)] + for t in trials: + runner.add_trial(t) + + runner.step() # run 1 + runner.step() # run 2 + assert all(t.status == Trial.RUNNING for t in trials) + + runner.step() # 1 result + print(runner.debug_string()) + + cluster.remove_node(node) + + runner.step() # recover + for i in range(5): + runner.step() + assert all(t.status == Trial.TERMINATED for t in trials) + + with pytest.raises(TuneError): + runner.step() + + +def test_trial_migration(start_connected_cluster): + """Removing a node should cause a Trial to be requeued.""" + cluster = start_connected_cluster + node = cluster.add_node(resources=dict(CPU=1)) + + runner = TrialRunner(BasicVariantGenerator()) + kwargs = { + "stopping_criterion": { + "training_iteration": 3 + } + } + + tune.register_trainable("test", _Train) + trials = [Trial("test", **kwargs), Trial("test", **kwargs)] + for t in trials: + runner.add_trial(t) + + runner.step() # run 1 + runner.step() # run 2 + assert all(t.status == Trial.RUNNING for t in trials) + + runner.step() # 1 result + print(runner.debug_string()) + + cluster.remove_node(node) + node2 = cluster.add_node(resources=dict(CPU=1)) + + runner.step() # recover + for i in range(5): + runner.step() + print(runner.debug_string()) + assert all(t.status == Trial.TERMINATED for t in trials) + + with pytest.raises(TuneError): + runner.step() diff --git a/test/jenkins_tests/run_multi_node_tests.sh b/test/jenkins_tests/run_multi_node_tests.sh index 380730fcb35f..e47ee01a0893 100755 --- a/test/jenkins_tests/run_multi_node_tests.sh +++ b/test/jenkins_tests/run_multi_node_tests.sh @@ -261,6 +261,9 @@ docker run --rm --shm-size=10G --memory=10G $DOCKER_SHA \ docker run --rm --shm-size=10G --memory=10G $DOCKER_SHA \ python /ray/python/ray/rllib/test/test_supported_spaces.py +docker run --rm --shm-size=10G --memory=10G $DOCKER_SHA \ + pytest /ray/python/ray/tune/test/cluster_tests.py + docker run --rm --shm-size=10G --memory=10G $DOCKER_SHA \ python /ray/python/ray/tune/examples/tune_mnist_ray.py \ --smoke-test From 32d1242b5b848176f4ef45e43fe59a8c0429e471 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 5 Nov 2018 00:04:27 -0800 Subject: [PATCH 03/74] Add cluster bookkeeping code --- python/ray/tune/test/cluster_tests.py | 34 +++++++++++++++------------ 1 file changed, 19 insertions(+), 15 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 57268b28ee69..065c8cac1d4b 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -42,14 +42,19 @@ def _restore(self, state): def test_counting_resources(start_connected_cluster): - """Removing a node should cause a Trial to be requeued.""" + """Tests that Tune accounting is consistent with actual cluster.""" cluster = start_connected_cluster - node = cluster.add_node(resources=dict(CPU=1)) + assert ray.global_state.cluster_resources()["CPU"] == 1 + nodes = [] + nodes += [cluster.add_node(resources=dict(CPU=1))] + cluster.wait_for_nodes() + assert ray.global_state.cluster_resources()["CPU"] == 2 + runner = TrialRunner(BasicVariantGenerator()) kwargs = { "stopping_criterion": { - "training_iteration": 3 + "training_iteration": 10 } } @@ -59,24 +64,23 @@ def test_counting_resources(start_connected_cluster): runner.add_trial(t) runner.step() # run 1 - cluster.remove_node(node) + cluster.remove_node(nodes.pop()) + cluster.wait_for_nodes() + assert ray.global_state.cluster_resources()["CPU"] == 1 runner.step() # run 2 - node = cluster.add_node(resources=dict(CPU=1)) - assert all(t.status == Trial.RUNNING for t in trials) + for i in range(5): + nodes += [cluster.add_node(resources=dict(CPU=1))] + cluster.wait_for_nodes() + assert ray.global_state.cluster_resources()["CPU"] == 6 runner.step() # 1 result - print(runner.debug_string()) - cluster.remove_node(node) - - runner.step() # recover for i in range(5): - runner.step() - assert all(t.status == Trial.TERMINATED for t in trials) - - with pytest.raises(TuneError): - runner.step() + node = nodes.pop() + cluster.remove_node(node) + cluster.wait_for_nodes() + assert ray.global_state.cluster_resources()["CPU"] == 1 def test_remove_node_before_result(start_connected_cluster): From 9ec3a60caaadc3d77f5a0dbdc3c3099ac72417b5 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 5 Nov 2018 00:18:08 -0800 Subject: [PATCH 04/74] add test for adding node --- python/ray/test/test_global_state.py | 31 +++++++++++++++++++++++++++- 1 file changed, 30 insertions(+), 1 deletion(-) diff --git a/python/ray/test/test_global_state.py b/python/ray/test/test_global_state.py index c5501dc9c525..33af028d836c 100644 --- a/python/ray/test/test_global_state.py +++ b/python/ray/test/test_global_state.py @@ -16,6 +16,17 @@ def ray_start(): # The code after the yield will run as teardown code. ray.shutdown() +@pytest.fixture +def cluster_start(): + # Start the Ray processes. + cluster = Cluster( + initialize_head=True, connect=True, + head_node_args={"resources": dict(CPU=1)}) + yield cluster + # The code after the yield will run as teardown code. + ray.shutdown() + cluster.shutdown() + def test_replenish_resources(ray_start): cluster_resources = ray.global_state.cluster_resources() @@ -34,7 +45,6 @@ def cpu_task(): while not resources_reset and time.time() - start < timeout: available_resources = ray.global_state.available_resources() resources_reset = (cluster_resources == available_resources) - assert resources_reset @@ -56,3 +66,22 @@ def cpu_task(): "CPU"] == cluster_resources["CPU"] - 1 assert resource_used + + +def test_proper_cluster_resources(cluster_start): + """Tests that Global State API is consistent with actual cluster.""" + cluster = cluster_start + assert ray.global_state.cluster_resources()["CPU"] == 1 + nodes = [] + nodes += [cluster.add_node(resources=dict(CPU=1))] + cluster.wait_for_nodes() + assert ray.global_state.cluster_resources()["CPU"] == 2 + + cluster.remove_node(nodes.pop()) + cluster.wait_for_nodes() + assert ray.global_state.cluster_resources()["CPU"] == 1 + + for i in range(5): + nodes += [cluster.add_node(resources=dict(CPU=1))] + cluster.wait_for_nodes() + assert ray.global_state.cluster_resources()["CPU"] == 6 From 44fe1e222523091233171acabd1657a0270cfd78 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 5 Nov 2018 15:12:55 -0800 Subject: [PATCH 05/74] multinode test fixes --- python/ray/test/cluster_utils.py | 4 +++- python/ray/test/test_global_state.py | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/python/ray/test/cluster_utils.py b/python/ray/test/cluster_utils.py index c4cf2b801623..fbce577035c8 100644 --- a/python/ray/test/cluster_utils.py +++ b/python/ray/test/cluster_utils.py @@ -114,11 +114,13 @@ def wait_for_nodes(self, retries=20): Args: retries (int): Number of times to retry checking client table. """ + for i in range(retries): if not ray.is_initialized() or not self._check_registered_nodes(): time.sleep(0.3) else: - break + return True + return False def _check_registered_nodes(self): registered = len([ diff --git a/python/ray/test/test_global_state.py b/python/ray/test/test_global_state.py index 33af028d836c..a4427283eed5 100644 --- a/python/ray/test/test_global_state.py +++ b/python/ray/test/test_global_state.py @@ -6,6 +6,7 @@ import time import ray +from ray.test.cluster_utils import Cluster @pytest.fixture @@ -16,6 +17,7 @@ def ray_start(): # The code after the yield will run as teardown code. ray.shutdown() + @pytest.fixture def cluster_start(): # Start the Ray processes. @@ -78,7 +80,7 @@ def test_proper_cluster_resources(cluster_start): assert ray.global_state.cluster_resources()["CPU"] == 2 cluster.remove_node(nodes.pop()) - cluster.wait_for_nodes() + cluster.wait_for_nodes(100) assert ray.global_state.cluster_resources()["CPU"] == 1 for i in range(5): From d9c9e3bc35cc9f506a9ad340d4a516c0f2d901eb Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 5 Nov 2018 16:02:35 -0800 Subject: [PATCH 06/74] First pass at allowing updatable values --- python/ray/scripts/scripts.py | 14 ++++++++--- python/ray/services.py | 45 +++++++++++++++++++++++----------- python/ray/worker.py | 10 +++++--- src/ray/ray_config.h | 24 +++++++++++++++++- src/ray/raylet/main.cc | 23 ++++++++++++++--- src/ray/raylet/monitor_main.cc | 5 ++-- 6 files changed, 94 insertions(+), 27 deletions(-) diff --git a/python/ray/scripts/scripts.py b/python/ray/scripts/scripts.py index 688fbb3ae480..73ed03112d73 100644 --- a/python/ray/scripts/scripts.py +++ b/python/ray/scripts/scripts.py @@ -193,13 +193,19 @@ def cli(logging_level, logging_format): "--temp-dir", default=None, help="manually specify the root temporary dir of the Ray process") +@click.option( + "-f", + "--config-file", + default=None, + type=str, + help="If specified, use config options from this file. ") def start(node_ip_address, redis_address, redis_port, num_redis_shards, redis_max_clients, redis_password, redis_shard_ports, object_manager_port, node_manager_port, object_store_memory, num_workers, num_cpus, num_gpus, resources, head, no_ui, block, plasma_directory, huge_pages, autoscaling_config, no_redirect_worker_output, no_redirect_output, - plasma_store_socket_name, raylet_socket_name, temp_dir): + plasma_store_socket_name, raylet_socket_name, temp_dir, config_file): # Convert hostnames to numerical IP address. if node_ip_address is not None: node_ip_address = services.address_to_ip(node_ip_address) @@ -269,7 +275,8 @@ def start(node_ip_address, redis_address, redis_port, num_redis_shards, autoscaling_config=autoscaling_config, plasma_store_socket_name=plasma_store_socket_name, raylet_socket_name=raylet_socket_name, - temp_dir=temp_dir) + temp_dir=temp_dir, + config_file=config_file) logger.info(address_info) logger.info( "\nStarted Ray on this node. You can add additional nodes to " @@ -348,7 +355,8 @@ def start(node_ip_address, redis_address, redis_port, num_redis_shards, huge_pages=huge_pages, plasma_store_socket_name=plasma_store_socket_name, raylet_socket_name=raylet_socket_name, - temp_dir=temp_dir) + temp_dir=temp_dir, + config_file=config_file) logger.info(address_info) logger.info("\nStarted Ray on this node. If you wish to terminate the " "processes that have been started, run\n\n" diff --git a/python/ray/services.py b/python/ray/services.py index 42e65809e997..b9ae8daa406c 100644 --- a/python/ray/services.py +++ b/python/ray/services.py @@ -857,6 +857,7 @@ def start_raylet(redis_address, stdout_file=None, stderr_file=None, cleanup=True, + config=None, redis_password=None): """Start a raylet, which is a combined local scheduler and object manager. @@ -892,6 +893,9 @@ def start_raylet(redis_address, Returns: The raylet socket name. """ + config = config or {} + config_str = ",".join(["{},{}".format(*kv) for kv in config.items()]) + if use_valgrind and use_profiler: raise Exception("Cannot use valgrind and profiler at the same time.") @@ -903,11 +907,8 @@ def start_raylet(redis_address, 1, min(multiprocessing.cpu_count(), static_resources["CPU"])) # Format the resource argument in a form like 'CPU,1.0,GPU,0,Custom,3'. - resource_argument = ",".join([ - "{},{}".format(resource_name, resource_value) - for resource_name, resource_value in zip(static_resources.keys(), - static_resources.values()) - ]) + resource_argument = ",".join( + ["{},{}".format(*kv) for kv in static_resources.items()]) gcs_ip_address, gcs_port = redis_address.split(":") @@ -945,6 +946,7 @@ def start_raylet(redis_address, str(num_workers), str(maximum_startup_concurrency), resource_argument, + config_str, start_worker_command, "", # Worker command for Java, not needed for Python. redis_password or "", @@ -1198,7 +1200,8 @@ def start_raylet_monitor(redis_address, stdout_file=None, stderr_file=None, cleanup=True, - redis_password=None): + redis_password=None, + config=None): """Run a process to monitor the other processes. Args: @@ -1215,7 +1218,9 @@ def start_raylet_monitor(redis_address, """ gcs_ip_address, gcs_port = redis_address.split(":") redis_password = redis_password or "" - command = [RAYLET_MONITOR_EXECUTABLE, gcs_ip_address, gcs_port] + config = config or {} + config_str = ",".join(["{},{}".format(*kv) for kv in config.items()]) + command = [RAYLET_MONITOR_EXECUTABLE, gcs_ip_address, gcs_port, config_str] p = subprocess.Popen(command, stdout=stdout_file, stderr=stderr_file) if cleanup: all_processes[PROCESS_TYPE_MONITOR].append(p) @@ -1246,7 +1251,8 @@ def start_ray_processes(address_info=None, autoscaling_config=None, plasma_store_socket_name=None, raylet_socket_name=None, - temp_dir=None): + temp_dir=None, + config_file=None): """Helper method to start Ray processes. Args: @@ -1322,6 +1328,11 @@ def start_ray_processes(address_info=None, logger.info("Process STDOUT and STDERR is being redirected to {}.".format( get_logs_dir_path())) + config = None + if config_file: + with open(config_file) as f: + config = yaml.load(f) + if resources is None: resources = {} if not isinstance(resources, list): @@ -1382,7 +1393,8 @@ def start_ray_processes(address_info=None, stdout_file=monitor_stdout_file, stderr_file=monitor_stderr_file, cleanup=cleanup, - redis_password=redis_password) + redis_password=redis_password, + config=config) if redis_shards == []: # Get redis shards from primary redis instance. redis_ip_address, redis_port = redis_address.split(":") @@ -1460,7 +1472,8 @@ def start_ray_processes(address_info=None, stdout_file=raylet_stdout_file, stderr_file=raylet_stderr_file, cleanup=cleanup, - redis_password=redis_password)) + redis_password=redis_password, + config=config)) # Try to start the web UI. if include_webui: @@ -1493,7 +1506,8 @@ def start_ray_node(node_ip_address, huge_pages=False, plasma_store_socket_name=None, raylet_socket_name=None, - temp_dir=None): + temp_dir=None, + config_file=None): """Start the Ray processes for a single node. This assumes that the Ray processes on some master node have already been @@ -1564,7 +1578,8 @@ def start_ray_node(node_ip_address, huge_pages=huge_pages, plasma_store_socket_name=plasma_store_socket_name, raylet_socket_name=raylet_socket_name, - temp_dir=temp_dir) + temp_dir=temp_dir, + config_file=config_file) def start_ray_head(address_info=None, @@ -1591,7 +1606,8 @@ def start_ray_head(address_info=None, autoscaling_config=None, plasma_store_socket_name=None, raylet_socket_name=None, - temp_dir=None): + temp_dir=None, + config_file=None): """Start Ray in local mode. Args: @@ -1684,4 +1700,5 @@ def start_ray_head(address_info=None, autoscaling_config=autoscaling_config, plasma_store_socket_name=plasma_store_socket_name, raylet_socket_name=raylet_socket_name, - temp_dir=temp_dir) + temp_dir=temp_dir, + config_file=config_file) diff --git a/python/ray/worker.py b/python/ray/worker.py index 8eee516ab119..066b4d2721a9 100644 --- a/python/ray/worker.py +++ b/python/ray/worker.py @@ -1282,7 +1282,8 @@ def _init(address_info=None, driver_id=None, plasma_store_socket_name=None, raylet_socket_name=None, - temp_dir=None): + temp_dir=None, + config_file=None): """Helper method to connect to an existing Ray cluster or start a new one. This method handles two cases. Either a Ray cluster already exists and we @@ -1416,7 +1417,8 @@ def _init(address_info=None, include_webui=include_webui, plasma_store_socket_name=plasma_store_socket_name, raylet_socket_name=raylet_socket_name, - temp_dir=temp_dir) + temp_dir=temp_dir, + config_file=config_file) else: if redis_address is None: raise Exception("When connecting to an existing cluster, " @@ -1519,6 +1521,7 @@ def init(redis_address=None, plasma_store_socket_name=None, raylet_socket_name=None, temp_dir=None, + config_file=None, use_raylet=None): """Connect to an existing Ray cluster or start one and connect to it. @@ -1647,7 +1650,8 @@ def init(redis_address=None, driver_id=driver_id, plasma_store_socket_name=plasma_store_socket_name, raylet_socket_name=raylet_socket_name, - temp_dir=temp_dir) + temp_dir=temp_dir, + config_file=config_file) for hook in _post_init_hooks: hook() return ret diff --git a/src/ray/ray_config.h b/src/ray/ray_config.h index 7a7ff6aa7689..692aa17c51f7 100644 --- a/src/ray/ray_config.h +++ b/src/ray/ray_config.h @@ -102,6 +102,23 @@ class RayConfig { int num_workers_per_process() const { return num_workers_per_process_; } + void initialize(const std::unordered_map &config_map) { + RAY_CHECK(!initialized_); + for (auto const &pair : config_map) { + switch (pair.first): + case "handler_warning_timeout_ms": { + handler_warning_timeout_ms_ = pair.second; + } break; + case "num_heartbeats_timeout": { + num_heartbeats_timeout_ = pair.second; + } break; + default: + RAY_LOG(FATAL) << "Invalid config key: " << pair.first; + } + } + initialized_ = true; + } + private: RayConfig() : ray_protocol_version_(0x0000000000000000), @@ -138,7 +155,8 @@ class RayConfig { object_manager_pull_timeout_ms_(100), object_manager_push_timeout_ms_(10000), object_manager_default_chunk_size_(1000000), - num_workers_per_process_(1) {} + num_workers_per_process_(1), + initialized_(false) {} ~RayConfig() {} @@ -263,6 +281,10 @@ class RayConfig { /// Number of workers per process int num_workers_per_process_; + + /// Whether the initialization of the instance has been called before. + /// The RayConfig instance can only (and must) be initialized once. + bool initialized_; }; #endif // RAY_CONFIG_H diff --git a/src/ray/raylet/main.cc b/src/ray/raylet/main.cc index 3e2483ee6b4e..be67f685e8da 100644 --- a/src/ray/raylet/main.cc +++ b/src/ray/raylet/main.cc @@ -20,7 +20,7 @@ int main(int argc, char *argv[]) { ray::RayLogLevel::INFO, /*log_dir=*/""); ray::RayLog::InstallFailureSignalHandler(); - RAY_CHECK(argc == 13 || argc == 14); + RAY_CHECK(argc == 14 || argc == 15); const std::string raylet_socket_name = std::string(argv[1]); const std::string store_socket_name = std::string(argv[2]); @@ -32,13 +32,27 @@ int main(int argc, char *argv[]) { int num_initial_workers = std::stoi(argv[8]); int maximum_startup_concurrency = std::stoi(argv[9]); const std::string static_resource_list = std::string(argv[10]); - const std::string python_worker_command = std::string(argv[11]); - const std::string java_worker_command = std::string(argv[12]); - const std::string redis_password = (argc == 14 ? std::string(argv[13]) : ""); + const std::string config_list = std::string(argv[11]); + const std::string python_worker_command = std::string(argv[12]); + const std::string java_worker_command = std::string(argv[13]); + const std::string redis_password = (argc == 15 ? std::string(argv[14]) : ""); // Configuration for the node manager. ray::raylet::NodeManagerConfig node_manager_config; std::unordered_map static_resource_conf; + std::unordered_map raylet_config; + + // Parse the configuration list. + std::istringstream config_string(config_list); + std::string config_name; + std::string config_value; + + while (std::getline(config_string, config_name, ',')) { + RAY_CHECK(std::getline(config_string, config_value, ',')); + // TODO(rkn): The line below could throw an exception. What should we do about this? + raylet_config[config_name] = std::stoi(config_value); + } + // Parse the resource list. std::istringstream resource_string(static_resource_list); std::string resource_name; @@ -49,6 +63,7 @@ int main(int argc, char *argv[]) { // TODO(rkn): The line below could throw an exception. What should we do about this? static_resource_conf[resource_name] = std::stod(resource_quantity); } + node_manager_config.resource_config = ray::raylet::ResourceSet(std::move(static_resource_conf)); RAY_LOG(DEBUG) << "Starting raylet with static resource configuration: " diff --git a/src/ray/raylet/monitor_main.cc b/src/ray/raylet/monitor_main.cc index f997566a5076..e1971981d4db 100644 --- a/src/ray/raylet/monitor_main.cc +++ b/src/ray/raylet/monitor_main.cc @@ -8,11 +8,12 @@ int main(int argc, char *argv[]) { ray::RayLog::ShutDownRayLog, argv[0], ray::RayLogLevel::INFO, /*log_dir=*/""); ray::RayLog::InstallFailureSignalHandler(); - RAY_CHECK(argc == 3 || argc == 4); + RAY_CHECK(argc == 4 || argc == 5); const std::string redis_address = std::string(argv[1]); int redis_port = std::stoi(argv[2]); - const std::string redis_password = (argc == 4 ? std::string(argv[3]) : ""); + const std::string config_list = std::string(argv[3]); + const std::string redis_password = (argc == 5 ? std::string(argv[4]) : ""); // Initialize the monitor. boost::asio::io_service io_service; From d6cade1520d7156ee575d6124130b6d8d443502d Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 5 Nov 2018 16:58:20 -0800 Subject: [PATCH 07/74] Fix compilation issues --- python/ray/services.py | 1 + src/ray/ray_config.h | 13 ++++++------- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/python/ray/services.py b/python/ray/services.py index b9ae8daa406c..fc90c9909507 100644 --- a/python/ray/services.py +++ b/python/ray/services.py @@ -14,6 +14,7 @@ import sys import threading import time +import yaml from collections import OrderedDict import redis diff --git a/src/ray/ray_config.h b/src/ray/ray_config.h index 692aa17c51f7..a5a31b08b9e4 100644 --- a/src/ray/ray_config.h +++ b/src/ray/ray_config.h @@ -1,7 +1,9 @@ #ifndef RAY_CONFIG_H #define RAY_CONFIG_H -#include +#include + +#include "ray/util/logging.h" class RayConfig { public: @@ -105,14 +107,11 @@ class RayConfig { void initialize(const std::unordered_map &config_map) { RAY_CHECK(!initialized_); for (auto const &pair : config_map) { - switch (pair.first): - case "handler_warning_timeout_ms": { + if (pair.first == "handler_warning_timeout_ms") { handler_warning_timeout_ms_ = pair.second; - } break; - case "num_heartbeats_timeout": { + } else if (pair.first == "num_heartbeats_timeout") { num_heartbeats_timeout_ = pair.second; - } break; - default: + } else { RAY_LOG(FATAL) << "Invalid config key: " << pair.first; } } From a95c718616214b0018ff37feb7c6afba0fae6f55 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 5 Nov 2018 17:30:27 -0800 Subject: [PATCH 08/74] Add config file parsing --- python/ray/test/test_global_state.py | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/python/ray/test/test_global_state.py b/python/ray/test/test_global_state.py index a4427283eed5..24d4bdc4a6cd 100644 --- a/python/ray/test/test_global_state.py +++ b/python/ray/test/test_global_state.py @@ -4,6 +4,9 @@ import pytest import time +import os +import yaml +import tempfile import ray from ray.test.cluster_utils import Cluster @@ -21,10 +24,19 @@ def ray_start(): @pytest.fixture def cluster_start(): # Start the Ray processes. + memory_config = yaml.dump({"num_heartbeats_timeout": 10}) + buff = tempfile.NamedTemporaryFile(mode='w+', delete=False) + path = buff.name + buff.write(memory_config) + buff.close() + import ipdb; ipdb.set_trace(context=5) cluster = Cluster( initialize_head=True, connect=True, - head_node_args={"resources": dict(CPU=1)}) + head_node_args={ + "resources": dict(CPU=1), + "config_file": path}) yield cluster + os.unlink(path) # The code after the yield will run as teardown code. ray.shutdown() cluster.shutdown() @@ -80,7 +92,7 @@ def test_proper_cluster_resources(cluster_start): assert ray.global_state.cluster_resources()["CPU"] == 2 cluster.remove_node(nodes.pop()) - cluster.wait_for_nodes(100) + cluster.wait_for_nodes() assert ray.global_state.cluster_resources()["CPU"] == 1 for i in range(5): From 5814655f802ee5bf5c9a8d32343f8cfa004001ec Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 5 Nov 2018 17:53:28 -0800 Subject: [PATCH 09/74] Full initialization --- src/ray/raylet/main.cc | 2 ++ src/ray/raylet/monitor_main.cc | 16 ++++++++++++++++ 2 files changed, 18 insertions(+) diff --git a/src/ray/raylet/main.cc b/src/ray/raylet/main.cc index be67f685e8da..93ccc28fdbe7 100644 --- a/src/ray/raylet/main.cc +++ b/src/ray/raylet/main.cc @@ -53,6 +53,8 @@ int main(int argc, char *argv[]) { raylet_config[config_name] = std::stoi(config_value); } + RayConfig::instance().initialize(raylet_config); + // Parse the resource list. std::istringstream resource_string(static_resource_list); std::string resource_name; diff --git a/src/ray/raylet/monitor_main.cc b/src/ray/raylet/monitor_main.cc index e1971981d4db..b12c3cfc2373 100644 --- a/src/ray/raylet/monitor_main.cc +++ b/src/ray/raylet/monitor_main.cc @@ -1,5 +1,6 @@ #include +#include "ray/ray_config.h" #include "ray/raylet/monitor.h" #include "ray/util/util.h" @@ -15,6 +16,21 @@ int main(int argc, char *argv[]) { const std::string config_list = std::string(argv[3]); const std::string redis_password = (argc == 5 ? std::string(argv[4]) : ""); + std::unordered_map raylet_config; + + // Parse the configuration list. + std::istringstream config_string(config_list); + std::string config_name; + std::string config_value; + + while (std::getline(config_string, config_name, ',')) { + RAY_CHECK(std::getline(config_string, config_value, ',')); + // TODO(rkn): The line below could throw an exception. What should we do about this? + raylet_config[config_name] = std::stoi(config_value); + } + + RayConfig::instance().initialize(raylet_config); + // Initialize the monitor. boost::asio::io_service io_service; ray::raylet::Monitor monitor(io_service, redis_address, redis_port, redis_password); From 282483606273bcd8493bc5ddd5562f2f08ad5ba9 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 5 Nov 2018 18:21:26 -0800 Subject: [PATCH 10/74] Wrote a good test --- python/ray/test/test_global_state.py | 28 +++++++++++++++++++--------- 1 file changed, 19 insertions(+), 9 deletions(-) diff --git a/python/ray/test/test_global_state.py b/python/ray/test/test_global_state.py index 24d4bdc4a6cd..2d64f627ec8c 100644 --- a/python/ray/test/test_global_state.py +++ b/python/ray/test/test_global_state.py @@ -3,6 +3,12 @@ from __future__ import print_function import pytest + +try: + import pytest_timeout +except ModuleNotFoundError as e: + pytest_timeout = None + import time import os import yaml @@ -29,7 +35,6 @@ def cluster_start(): path = buff.name buff.write(memory_config) buff.close() - import ipdb; ipdb.set_trace(context=5) cluster = Cluster( initialize_head=True, connect=True, head_node_args={ @@ -42,6 +47,10 @@ def cluster_start(): cluster.shutdown() +# TODO(rliaw): The proper way to do this is to have the pytest config setup. +@pytest.mark.skipif(pytest_timeout==None, reason="Timeout package"\ + " not installed; skipping test that may hang.") +@pytest.mark.timeout(10) def test_replenish_resources(ray_start): cluster_resources = ray.global_state.cluster_resources() available_resources = ray.global_state.available_resources() @@ -52,19 +61,19 @@ def cpu_task(): pass ray.get(cpu_task.remote()) - start = time.time() resources_reset = False - timeout = 10 - while not resources_reset and time.time() - start < timeout: + while not resources_reset: available_resources = ray.global_state.available_resources() resources_reset = (cluster_resources == available_resources) assert resources_reset +@pytest.mark.skipif(pytest_timeout==None, reason="Timeout package"\ + " not installed; skipping test that may hang.") +@pytest.mark.timeout(10) def test_uses_resources(ray_start): cluster_resources = ray.global_state.cluster_resources() - @ray.remote def cpu_task(): time.sleep(1) @@ -72,9 +81,7 @@ def cpu_task(): cpu_task.remote() resource_used = False - start = time.time() - timeout = 10 - while not resource_used and time.time() - start < timeout: + while not resource_used: available_resources = ray.global_state.available_resources() resource_used = available_resources[ "CPU"] == cluster_resources["CPU"] - 1 @@ -82,7 +89,10 @@ def cpu_task(): assert resource_used -def test_proper_cluster_resources(cluster_start): +@pytest.mark.skipif(pytest_timeout==None, reason="Timeout package"\ + " not installed; skipping test that may hang.") +@pytest.mark.timeout(20) +def test_add_remove_cluster_resources(cluster_start): """Tests that Global State API is consistent with actual cluster.""" cluster = cluster_start assert ray.global_state.cluster_resources()["CPU"] == 1 From 4842481e87c8b90a53d47a5a4382bf7bd74421ec Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 6 Nov 2018 17:19:40 -0800 Subject: [PATCH 11/74] configuration parsing and stuff --- python/ray/scripts/scripts.py | 11 +++++------ python/ray/services.py | 22 ++++++++++++---------- 2 files changed, 17 insertions(+), 16 deletions(-) diff --git a/python/ray/scripts/scripts.py b/python/ray/scripts/scripts.py index 73ed03112d73..c674f26cd376 100644 --- a/python/ray/scripts/scripts.py +++ b/python/ray/scripts/scripts.py @@ -194,18 +194,17 @@ def cli(logging_level, logging_format): default=None, help="manually specify the root temporary dir of the Ray process") @click.option( - "-f", - "--config-file", + "--internal-config", default=None, type=str, - help="If specified, use config options from this file. ") + help="Do NOT use this. This is for debugging and development purposes ONLY.") def start(node_ip_address, redis_address, redis_port, num_redis_shards, redis_max_clients, redis_password, redis_shard_ports, object_manager_port, node_manager_port, object_store_memory, num_workers, num_cpus, num_gpus, resources, head, no_ui, block, plasma_directory, huge_pages, autoscaling_config, no_redirect_worker_output, no_redirect_output, - plasma_store_socket_name, raylet_socket_name, temp_dir, config_file): + plasma_store_socket_name, raylet_socket_name, temp_dir, internal_config): # Convert hostnames to numerical IP address. if node_ip_address is not None: node_ip_address = services.address_to_ip(node_ip_address) @@ -276,7 +275,7 @@ def start(node_ip_address, redis_address, redis_port, num_redis_shards, plasma_store_socket_name=plasma_store_socket_name, raylet_socket_name=raylet_socket_name, temp_dir=temp_dir, - config_file=config_file) + internal_config=internal_config) logger.info(address_info) logger.info( "\nStarted Ray on this node. You can add additional nodes to " @@ -356,7 +355,7 @@ def start(node_ip_address, redis_address, redis_port, num_redis_shards, plasma_store_socket_name=plasma_store_socket_name, raylet_socket_name=raylet_socket_name, temp_dir=temp_dir, - config_file=config_file) + internal_config=internal_config) logger.info(address_info) logger.info("\nStarted Ray on this node. If you wish to terminate the " "processes that have been started, run\n\n" diff --git a/python/ray/services.py b/python/ray/services.py index fc90c9909507..8569b7d9510a 100644 --- a/python/ray/services.py +++ b/python/ray/services.py @@ -14,7 +14,6 @@ import sys import threading import time -import yaml from collections import OrderedDict import redis @@ -1253,7 +1252,7 @@ def start_ray_processes(address_info=None, plasma_store_socket_name=None, raylet_socket_name=None, temp_dir=None, - config_file=None): + internal_config=None): """Helper method to start Ray processes. Args: @@ -1318,6 +1317,8 @@ def start_ray_processes(address_info=None, used by the raylet process. temp_dir (str): If provided, it will specify the root temporary directory for the Ray process. + internal_config (str): JSON configuration for overriding + RayConfig defaults. For testing purposes ONLY. Returns: A dictionary of the address information for the processes that were @@ -1329,10 +1330,7 @@ def start_ray_processes(address_info=None, logger.info("Process STDOUT and STDERR is being redirected to {}.".format( get_logs_dir_path())) - config = None - if config_file: - with open(config_file) as f: - config = yaml.load(f) + config = json.loads(internal_config) if internal_config else None if resources is None: resources = {} @@ -1508,7 +1506,7 @@ def start_ray_node(node_ip_address, plasma_store_socket_name=None, raylet_socket_name=None, temp_dir=None, - config_file=None): + internal_config=None): """Start the Ray processes for a single node. This assumes that the Ray processes on some master node have already been @@ -1552,6 +1550,8 @@ def start_ray_node(node_ip_address, used by the raylet process. temp_dir (str): If provided, it will specify the root temporary directory for the Ray process. + internal_config (str): JSON configuration for overriding + RayConfig defaults. For testing purposes ONLY. Returns: A dictionary of the address information for the processes that were @@ -1580,7 +1580,7 @@ def start_ray_node(node_ip_address, plasma_store_socket_name=plasma_store_socket_name, raylet_socket_name=raylet_socket_name, temp_dir=temp_dir, - config_file=config_file) + internal_config=internal_config) def start_ray_head(address_info=None, @@ -1608,7 +1608,7 @@ def start_ray_head(address_info=None, plasma_store_socket_name=None, raylet_socket_name=None, temp_dir=None, - config_file=None): + internal_config=None): """Start Ray in local mode. Args: @@ -1669,6 +1669,8 @@ def start_ray_head(address_info=None, used by the raylet process. temp_dir (str): If provided, it will specify the root temporary directory for the Ray process. + internal_config (str): JSON configuration for overriding + RayConfig defaults. For testing purposes ONLY. Returns: A dictionary of the address information for the processes that were @@ -1702,4 +1704,4 @@ def start_ray_head(address_info=None, plasma_store_socket_name=plasma_store_socket_name, raylet_socket_name=raylet_socket_name, temp_dir=temp_dir, - config_file=config_file) + internal_config=internal_config) From 8e521034772799e3d5f37b76ff4a7cd12dc7bea6 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 6 Nov 2018 17:41:17 -0800 Subject: [PATCH 12/74] docs --- python/ray/services.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/python/ray/services.py b/python/ray/services.py index 8569b7d9510a..3888ec883cfd 100644 --- a/python/ray/services.py +++ b/python/ray/services.py @@ -888,6 +888,8 @@ def start_raylet(redis_address, cleanup (bool): True if using Ray in local mode. If cleanup is true, then this process will be killed by serices.cleanup() when the Python process that imported services exits. + config (dict|None): Optional Raylet configuration that will + override defaults in RayConfig. redis_password (str): The password of the redis server. Returns: @@ -1215,6 +1217,8 @@ def start_raylet_monitor(redis_address, Python process that imported services exits. This is True by default. redis_password (str): The password of the redis server. + config (dict|None): Optional configuration that will + override defaults in RayConfig. """ gcs_ip_address, gcs_port = redis_address.split(":") redis_password = redis_password or "" From 83d6947e280cf4ddac5124b3020abcef20c1acdd Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 6 Nov 2018 18:11:56 -0800 Subject: [PATCH 13/74] write some tests, make it good --- python/ray/test/cluster_utils.py | 6 +++-- test/multi_node_test_2.py | 38 +++++++++++++++++++++++++++++++- 2 files changed, 41 insertions(+), 3 deletions(-) diff --git a/python/ray/test/cluster_utils.py b/python/ray/test/cluster_utils.py index c4cf2b801623..befa87ee93db 100644 --- a/python/ray/test/cluster_utils.py +++ b/python/ray/test/cluster_utils.py @@ -108,15 +108,17 @@ def remove_node(self, node): assert not node.any_processes_alive(), ( "There are zombie processes left over after killing.") - def wait_for_nodes(self, retries=20): + def wait_for_nodes(self, retries=30): """Waits for all nodes to be registered with global state. + By default, waits for 3 seconds. + Args: retries (int): Number of times to retry checking client table. """ for i in range(retries): if not ray.is_initialized() or not self._check_registered_nodes(): - time.sleep(0.3) + time.sleep(0.1) else: break diff --git a/test/multi_node_test_2.py b/test/multi_node_test_2.py index b0696f4b3285..bae4d79bff1a 100644 --- a/test/multi_node_test_2.py +++ b/test/multi_node_test_2.py @@ -4,6 +4,7 @@ import logging import pytest +import json import ray import ray.services as services @@ -15,7 +16,15 @@ @pytest.fixture def start_connected_cluster(): # Start the Ray processes. - g = Cluster(initialize_head=True, connect=True) + g = Cluster( + initialize_head=True, + connect=True, + head_node_args={ + "resources": dict(CPU=1), + "internal_config": json.dumps({ + "num_heartbeats_timeout": 10 + }) + }) yield g # The code after the yield will run as teardown code. ray.shutdown() @@ -34,6 +43,30 @@ def test_cluster(): assert not any(node.any_processes_alive() for node in g.list_all_nodes()) +def test_internal_config(): + cluster = Cluster( + initialize_head=True, + connect=True, + head_node_args={ + "resources": dict(CPU=1), + "internal_config": json.dumps({ + "num_heartbeats_timeout": 20 + }) + }) + worker = cluster.add_node() + cluster.wait_for_nodes() + + cluster.remove_node(worker) + cluster.wait_for_nodes(retries=10) + assert ray.global_state.cluster_resources()["CPU"] == 2 + + cluster.wait_for_nodes(retries=20) + assert ray.global_state.cluster_resources()["CPU"] == 1 + # The code after the yield will run as teardown code. + ray.shutdown() + cluster.shutdown() + + def test_wait_for_nodes(start_connected_cluster): """Unit test for `Cluster.wait_for_nodes`. @@ -45,10 +78,13 @@ def test_wait_for_nodes(start_connected_cluster): cluster.wait_for_nodes() [cluster.remove_node(w) for w in workers] cluster.wait_for_nodes() + + assert ray.global_state.cluster_resources()["CPU"] == 1 worker2 = cluster.add_node() cluster.wait_for_nodes() cluster.remove_node(worker2) cluster.wait_for_nodes() + assert ray.global_state.cluster_resources()["CPU"] == 1 def test_worker_plasma_store_failure(start_connected_cluster): From 80789674efe1ef4e17c37dc191e0b03822b9b95e Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 6 Nov 2018 19:20:06 -0800 Subject: [PATCH 14/74] fixed init --- python/ray/worker.py | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/python/ray/worker.py b/python/ray/worker.py index 201256c58e6f..86ded3a36661 100644 --- a/python/ray/worker.py +++ b/python/ray/worker.py @@ -1292,7 +1292,7 @@ def _init(address_info=None, plasma_store_socket_name=None, raylet_socket_name=None, temp_dir=None, - config_file=None): + internal_config=None): """Helper method to connect to an existing Ray cluster or start a new one. This method handles two cases. Either a Ray cluster already exists and we @@ -1354,6 +1354,8 @@ def _init(address_info=None, used by the raylet process. temp_dir (str): If provided, it will specify the root temporary directory for the Ray process. + internal_config (str): JSON configuration for overriding + RayConfig defaults. For testing purposes ONLY. Returns: Address information about the started processes. @@ -1427,7 +1429,7 @@ def _init(address_info=None, plasma_store_socket_name=plasma_store_socket_name, raylet_socket_name=raylet_socket_name, temp_dir=temp_dir, - config_file=config_file) + internal_config=internal_config) else: if redis_address is None: raise Exception("When connecting to an existing cluster, " @@ -1530,7 +1532,7 @@ def init(redis_address=None, plasma_store_socket_name=None, raylet_socket_name=None, temp_dir=None, - config_file=None, + internal_config=None, use_raylet=None): """Connect to an existing Ray cluster or start one and connect to it. @@ -1602,6 +1604,8 @@ def init(redis_address=None, used by the raylet process. temp_dir (str): If provided, it will specify the root temporary directory for the Ray process. + internal_config (str): JSON configuration for overriding + RayConfig defaults. For testing purposes ONLY. Returns: Address information about the started processes. @@ -1660,7 +1664,7 @@ def init(redis_address=None, plasma_store_socket_name=plasma_store_socket_name, raylet_socket_name=raylet_socket_name, temp_dir=temp_dir, - config_file=config_file) + internal_config=internal_config) for hook in _post_init_hooks: hook() return ret From 2db9f18436331bdc52ed8e4a8025d279106ff93a Mon Sep 17 00:00:00 2001 From: Robert Nishihara Date: Tue, 6 Nov 2018 22:48:14 -0800 Subject: [PATCH 15/74] Add all config options and bring back stress tests. --- python/ray/worker.py | 9 ++++-- src/ray/ray_config.h | 70 ++++++++++++++++++++++++++++++++++++++++++-- test/stress_tests.py | 9 +++--- 3 files changed, 79 insertions(+), 9 deletions(-) diff --git a/python/ray/worker.py b/python/ray/worker.py index 86ded3a36661..1aa8ff6aa04f 100644 --- a/python/ray/worker.py +++ b/python/ray/worker.py @@ -1292,7 +1292,7 @@ def _init(address_info=None, plasma_store_socket_name=None, raylet_socket_name=None, temp_dir=None, - internal_config=None): + _internal_config=None): """Helper method to connect to an existing Ray cluster or start a new one. This method handles two cases. Either a Ray cluster already exists and we @@ -1354,7 +1354,7 @@ def _init(address_info=None, used by the raylet process. temp_dir (str): If provided, it will specify the root temporary directory for the Ray process. - internal_config (str): JSON configuration for overriding + _internal_config (str): JSON configuration for overriding RayConfig defaults. For testing purposes ONLY. Returns: @@ -1429,7 +1429,7 @@ def _init(address_info=None, plasma_store_socket_name=plasma_store_socket_name, raylet_socket_name=raylet_socket_name, temp_dir=temp_dir, - internal_config=internal_config) + internal_config=_internal_config) else: if redis_address is None: raise Exception("When connecting to an existing cluster, " @@ -1470,6 +1470,9 @@ def _init(address_info=None, if raylet_socket_name is not None: raise Exception("When connecting to an existing cluster, " "raylet_socket_name must not be provided.") + if _internal_config is not None: + raise Exception("When connecting to an existing cluster, " + "_internal_config must not be provided.") # Get the node IP address if one is not provided. if node_ip_address is None: diff --git a/src/ray/ray_config.h b/src/ray/ray_config.h index 94a8d8a60f6f..afa52abfd911 100644 --- a/src/ray/ray_config.h +++ b/src/ray/ray_config.h @@ -107,12 +107,78 @@ class RayConfig { void initialize(const std::unordered_map &config_map) { RAY_CHECK(!initialized_); for (auto const &pair : config_map) { - if (pair.first == "handler_warning_timeout_ms") { + // We use a big chain of if else statements because C++ doesn't allow + // switch statements on strings. + if (pair.first == "ray_protocol_version") { + ray_protocol_version_ = pair.second; + } else if (pair.first == "handler_warning_timeout_ms") { handler_warning_timeout_ms_ = pair.second; + } else if (pair.first == "heartbeat_timeout_milliseconds") { + heartbeat_timeout_milliseconds_ = pair.second; } else if (pair.first == "num_heartbeats_timeout") { num_heartbeats_timeout_ = pair.second; + } else if (pair.first == "num_heartbeats_warning") { + num_heartbeats_warning_ = pair.second; + } else if (pair.first == "initial_reconstruction_timeout_milliseconds") { + initial_reconstruction_timeout_milliseconds_ = pair.second; + } else if (pair.first == "get_timeout_milliseconds") { + get_timeout_milliseconds_ = pair.second; + } else if (pair.first == "worker_get_request_size") { + worker_get_request_size_ = pair.second; + } else if (pair.first == "worker_fetch_request_size") { + worker_fetch_request_size_ = pair.second; + } else if (pair.first == "max_lineage_size") { + max_lineage_size_ = pair.second; + } else if (pair.first == "actor_max_dummy_objects") { + actor_max_dummy_objects_ = pair.second; + } else if (pair.first == "num_connect_attempts") { + num_connect_attempts_ = pair.second; + } else if (pair.first == "connect_timeout_milliseconds") { + connect_timeout_milliseconds_ = pair.second; + } else if (pair.first == "local_scheduler_fetch_timeout_milliseconds") { + local_scheduler_fetch_timeout_milliseconds_ = pair.second; + } else if (pair.first == "local_scheduler_reconstruction_timeout_milliseconds") { + local_scheduler_reconstruction_timeout_milliseconds_ = pair.second; + } else if (pair.first == "max_num_to_reconstruct") { + max_num_to_reconstruct_ = pair.second; + } else if (pair.first == "local_scheduler_fetch_request_size") { + local_scheduler_fetch_request_size_ = pair.second; + } else if (pair.first == "kill_worker_timeout_milliseconds") { + kill_worker_timeout_milliseconds_ = pair.second; + } else if (pair.first == "manager_timeout_milliseconds") { + manager_timeout_milliseconds_ = pair.second; + } else if (pair.first == "buf_size") { + buf_size_ = pair.second; + } else if (pair.first == "max_time_for_handler_milliseconds") { + max_time_for_handler_milliseconds_ = pair.second; + } else if (pair.first == "size_limit") { + size_limit_ = pair.second; + } else if (pair.first == "num_elements_limit") { + num_elements_limit_ = pair.second; + } else if (pair.first == "max_time_for_loop") { + max_time_for_loop_ = pair.second; + } else if (pair.first == "redis_db_connect_retries") { + redis_db_connect_retries_ = pair.second; + } else if (pair.first == "redis_db_connect_wait_milliseconds") { + redis_db_connect_wait_milliseconds_ = pair.second; + } else if (pair.first == "plasma_default_release_delay") { + plasma_default_release_delay_ = pair.second; + } else if (pair.first == "L3_cache_size_bytes") { + L3_cache_size_bytes_ = pair.second; + } else if (pair.first == "max_tasks_to_spillback") { + max_tasks_to_spillback_ = pair.second; + } else if (pair.first == "actor_creation_num_spillbacks_warning") { + actor_creation_num_spillbacks_warning_ = pair.second; + } else if (pair.first == "node_manager_forward_task_retry_timeout_milliseconds") { + node_manager_forward_task_retry_timeout_milliseconds_ = pair.second; + } else if (pair.first == "object_manager_pull_timeout_ms") { + object_manager_pull_timeout_ms_ = pair.second; + } else if (pair.first == "object_manager_push_timeout_ms") { + object_manager_push_timeout_ms_ = pair.second; + } else if (pair.first == "object_manager_default_chunk_size") { + object_manager_default_chunk_size_ = pair.second; } else { - RAY_LOG(FATAL) << "Invalid config key: " << pair.first; + RAY_LOG(FATAL) << "Received unexpected config parameter " << pair.first; } } initialized_ = true; diff --git a/test/stress_tests.py b/test/stress_tests.py index 307ae52e5531..284bdb4361fd 100644 --- a/test/stress_tests.py +++ b/test/stress_tests.py @@ -2,6 +2,7 @@ from __future__ import division from __future__ import print_function +import json import numpy as np import os import pytest @@ -216,7 +217,10 @@ def ray_start_reconstruction(request): start_ray_local=True, num_local_schedulers=num_local_schedulers, num_cpus=[1] * num_local_schedulers, - redirect_output=True) + redirect_output=True, + _internal_config=json.dumps({ + "initial_reconstruction_timeout_milliseconds": 200 + })) yield (redis_ip_address, redis_port, plasma_store_memory, num_local_schedulers) @@ -249,7 +253,6 @@ def ray_start_reconstruction(request): ray.shutdown() -@pytest.mark.skip("Add this test back once reconstruction is faster.") @pytest.mark.skipif( os.environ.get("RAY_USE_NEW_GCS") == "on", reason="Failing with new GCS API on Linux.") @@ -291,7 +294,6 @@ def foo(i, size): del values -@pytest.mark.skip("Add this test back once reconstruction is faster.") @pytest.mark.skipif( os.environ.get("RAY_USE_NEW_GCS") == "on", reason="Failing with new GCS API on Linux.") @@ -348,7 +350,6 @@ def single_dependency(i, arg): del values -@pytest.mark.skip("Add this test back once reconstruction is faster.") @pytest.mark.skipif( os.environ.get("RAY_USE_NEW_GCS") == "on", reason="Failing with new GCS API on Linux.") From 59480dc27414aa1b9b02abb6500ef017168db267 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 6 Nov 2018 23:38:19 -0800 Subject: [PATCH 16/74] Update python/ray/worker.py --- python/ray/worker.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/worker.py b/python/ray/worker.py index 1aa8ff6aa04f..7091d4314bdd 100644 --- a/python/ray/worker.py +++ b/python/ray/worker.py @@ -1667,7 +1667,7 @@ def init(redis_address=None, plasma_store_socket_name=plasma_store_socket_name, raylet_socket_name=raylet_socket_name, temp_dir=temp_dir, - internal_config=internal_config) + _internal_config=_internal_config) for hook in _post_init_hooks: hook() return ret From 6fa9d7c255b640114ab04b7c020121a165126826 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 6 Nov 2018 23:38:25 -0800 Subject: [PATCH 17/74] Update python/ray/worker.py --- python/ray/worker.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/worker.py b/python/ray/worker.py index 7091d4314bdd..a7b48163a06f 100644 --- a/python/ray/worker.py +++ b/python/ray/worker.py @@ -1607,7 +1607,7 @@ def init(redis_address=None, used by the raylet process. temp_dir (str): If provided, it will specify the root temporary directory for the Ray process. - internal_config (str): JSON configuration for overriding + _internal_config (str): JSON configuration for overriding RayConfig defaults. For testing purposes ONLY. Returns: From 856547cc0a09f23f6aa4008e1334e21f049554cb Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 6 Nov 2018 23:39:34 -0800 Subject: [PATCH 18/74] TEMP --- python/ray/tune/test/cluster_tests.py | 22 ++++++++++++++++++++-- 1 file changed, 20 insertions(+), 2 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 065c8cac1d4b..0415834ef759 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -2,7 +2,14 @@ from __future__ import division from __future__ import print_function +import os +import json +import tempfile import pytest +try: + import pytest_timeout +except ModuleNotFoundError as e: + pytest_timeout = None from ray.test.cluster_utils import Cluster import ray @@ -17,10 +24,15 @@ @pytest.fixture def start_connected_cluster(): # Start the Ray processes. + cluster = Cluster( initialize_head=True, connect=True, - head_node_args={"resources": dict(CPU=1)}) + head_node_args={ + "resources": dict(CPU=1), + "internal_config": json.dumps( + {"num_heartbeats_timeout": 10})}) yield cluster + os.unlink(path) # The code after the yield will run as teardown code. ray.shutdown() cluster.shutdown() @@ -83,6 +95,9 @@ def test_counting_resources(start_connected_cluster): assert ray.global_state.cluster_resources()["CPU"] == 1 +@pytest.mark.skipif(pytest_timeout==None, reason="Timeout package"\ + " not installed; skipping test that may hang.") +@pytest.mark.timeout(10, method="thread") def test_remove_node_before_result(start_connected_cluster): """Removing a node should cause a Trial to be requeued.""" cluster = start_connected_cluster @@ -108,7 +123,9 @@ def test_remove_node_before_result(start_connected_cluster): print(runner.debug_string()) cluster.remove_node(node) - + cluster.wait_for_nodes() + print("\n"*5) + print("!!!!!!!!!!!!!!!! Finished") runner.step() # recover for i in range(5): runner.step() @@ -143,6 +160,7 @@ def test_trial_migration(start_connected_cluster): print(runner.debug_string()) cluster.remove_node(node) + cluster.wait_for_nodes() node2 = cluster.add_node(resources=dict(CPU=1)) runner.step() # recover From 25e45cd2bd0e3554e1b742839c939a4a4959b342 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 6 Nov 2018 23:48:13 -0800 Subject: [PATCH 19/74] Fix internalization --- python/ray/scripts/scripts.py | 9 +++++---- python/ray/services.py | 18 +++++++++--------- python/ray/worker.py | 8 ++++---- 3 files changed, 18 insertions(+), 17 deletions(-) diff --git a/python/ray/scripts/scripts.py b/python/ray/scripts/scripts.py index 175449420e77..b646c58eb11b 100644 --- a/python/ray/scripts/scripts.py +++ b/python/ray/scripts/scripts.py @@ -197,14 +197,15 @@ def cli(logging_level, logging_format): "--internal-config", default=None, type=str, - help="Do NOT use this. This is for debugging and development purposes ONLY.") + help="Do NOT use this. This is for debugging/development purposes ONLY.") def start(node_ip_address, redis_address, redis_port, num_redis_shards, redis_max_clients, redis_password, redis_shard_ports, object_manager_port, node_manager_port, object_store_memory, num_workers, num_cpus, num_gpus, resources, head, no_ui, block, plasma_directory, huge_pages, autoscaling_config, no_redirect_worker_output, no_redirect_output, - plasma_store_socket_name, raylet_socket_name, temp_dir, internal_config): + plasma_store_socket_name, raylet_socket_name, + temp_dir, internal_config): # Convert hostnames to numerical IP address. if node_ip_address is not None: node_ip_address = services.address_to_ip(node_ip_address) @@ -275,7 +276,7 @@ def start(node_ip_address, redis_address, redis_port, num_redis_shards, plasma_store_socket_name=plasma_store_socket_name, raylet_socket_name=raylet_socket_name, temp_dir=temp_dir, - internal_config=internal_config) + _internal_config=internal_config) logger.info(address_info) logger.info( "\nStarted Ray on this node. You can add additional nodes to " @@ -355,7 +356,7 @@ def start(node_ip_address, redis_address, redis_port, num_redis_shards, plasma_store_socket_name=plasma_store_socket_name, raylet_socket_name=raylet_socket_name, temp_dir=temp_dir, - internal_config=internal_config) + _internal_config=internal_config) logger.info(address_info) logger.info("\nStarted Ray on this node. If you wish to terminate the " "processes that have been started, run\n\n" diff --git a/python/ray/services.py b/python/ray/services.py index f59e426514a3..cf0869fd0bbc 100644 --- a/python/ray/services.py +++ b/python/ray/services.py @@ -1269,7 +1269,7 @@ def start_ray_processes(address_info=None, plasma_store_socket_name=None, raylet_socket_name=None, temp_dir=None, - internal_config=None): + _internal_config=None): """Helper method to start Ray processes. Args: @@ -1334,7 +1334,7 @@ def start_ray_processes(address_info=None, used by the raylet process. temp_dir (str): If provided, it will specify the root temporary directory for the Ray process. - internal_config (str): JSON configuration for overriding + _internal_config (str): JSON configuration for overriding RayConfig defaults. For testing purposes ONLY. Returns: @@ -1347,7 +1347,7 @@ def start_ray_processes(address_info=None, logger.info("Process STDOUT and STDERR is being redirected to {}.".format( get_logs_dir_path())) - config = json.loads(internal_config) if internal_config else None + config = json.loads(_internal_config) if _internal_config else None if resources is None: resources = {} @@ -1523,7 +1523,7 @@ def start_ray_node(node_ip_address, plasma_store_socket_name=None, raylet_socket_name=None, temp_dir=None, - internal_config=None): + _internal_config=None): """Start the Ray processes for a single node. This assumes that the Ray processes on some master node have already been @@ -1567,7 +1567,7 @@ def start_ray_node(node_ip_address, used by the raylet process. temp_dir (str): If provided, it will specify the root temporary directory for the Ray process. - internal_config (str): JSON configuration for overriding + _internal_config (str): JSON configuration for overriding RayConfig defaults. For testing purposes ONLY. Returns: @@ -1597,7 +1597,7 @@ def start_ray_node(node_ip_address, plasma_store_socket_name=plasma_store_socket_name, raylet_socket_name=raylet_socket_name, temp_dir=temp_dir, - internal_config=internal_config) + _internal_config=_internal_config) def start_ray_head(address_info=None, @@ -1625,7 +1625,7 @@ def start_ray_head(address_info=None, plasma_store_socket_name=None, raylet_socket_name=None, temp_dir=None, - internal_config=None): + _internal_config=None): """Start Ray in local mode. Args: @@ -1686,7 +1686,7 @@ def start_ray_head(address_info=None, used by the raylet process. temp_dir (str): If provided, it will specify the root temporary directory for the Ray process. - internal_config (str): JSON configuration for overriding + _internal_config (str): JSON configuration for overriding RayConfig defaults. For testing purposes ONLY. Returns: @@ -1721,4 +1721,4 @@ def start_ray_head(address_info=None, plasma_store_socket_name=plasma_store_socket_name, raylet_socket_name=raylet_socket_name, temp_dir=temp_dir, - internal_config=internal_config) + _internal_config=_internal_config) diff --git a/python/ray/worker.py b/python/ray/worker.py index 1aa8ff6aa04f..2c44a8bf0fb8 100644 --- a/python/ray/worker.py +++ b/python/ray/worker.py @@ -1429,7 +1429,7 @@ def _init(address_info=None, plasma_store_socket_name=plasma_store_socket_name, raylet_socket_name=raylet_socket_name, temp_dir=temp_dir, - internal_config=_internal_config) + _internal_config=_internal_config) else: if redis_address is None: raise Exception("When connecting to an existing cluster, " @@ -1535,7 +1535,7 @@ def init(redis_address=None, plasma_store_socket_name=None, raylet_socket_name=None, temp_dir=None, - internal_config=None, + _internal_config=None, use_raylet=None): """Connect to an existing Ray cluster or start one and connect to it. @@ -1607,7 +1607,7 @@ def init(redis_address=None, used by the raylet process. temp_dir (str): If provided, it will specify the root temporary directory for the Ray process. - internal_config (str): JSON configuration for overriding + _internal_config (str): JSON configuration for overriding RayConfig defaults. For testing purposes ONLY. Returns: @@ -1667,7 +1667,7 @@ def init(redis_address=None, plasma_store_socket_name=plasma_store_socket_name, raylet_socket_name=raylet_socket_name, temp_dir=temp_dir, - internal_config=internal_config) + _internal_config=_internal_config) for hook in _post_init_hooks: hook() return ret From d3fa8f0da11134fde798deb6a39c8976a5112282 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 6 Nov 2018 23:51:38 -0800 Subject: [PATCH 20/74] some last changes --- test/multi_node_test_2.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/test/multi_node_test_2.py b/test/multi_node_test_2.py index bae4d79bff1a..f56ddd2e18b8 100644 --- a/test/multi_node_test_2.py +++ b/test/multi_node_test_2.py @@ -21,7 +21,7 @@ def start_connected_cluster(): connect=True, head_node_args={ "resources": dict(CPU=1), - "internal_config": json.dumps({ + "_internal_config": json.dumps({ "num_heartbeats_timeout": 10 }) }) @@ -49,7 +49,7 @@ def test_internal_config(): connect=True, head_node_args={ "resources": dict(CPU=1), - "internal_config": json.dumps({ + "_internal_config": json.dumps({ "num_heartbeats_timeout": 20 }) }) From c3c1c9c05f22469c19ae75b719bf2ebf5422d489 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 7 Nov 2018 00:25:40 -0800 Subject: [PATCH 21/74] skip for now --- python/ray/tune/test/cluster_tests.py | 8 ++++---- test/multi_node_test_2.py | 1 + 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 0415834ef759..d87402f26b3d 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -29,7 +29,7 @@ def start_connected_cluster(): initialize_head=True, connect=True, head_node_args={ "resources": dict(CPU=1), - "internal_config": json.dumps( + "_internal_config": json.dumps( {"num_heartbeats_timeout": 10})}) yield cluster os.unlink(path) @@ -95,9 +95,9 @@ def test_counting_resources(start_connected_cluster): assert ray.global_state.cluster_resources()["CPU"] == 1 +# @pytest.mark.timeout(10, method="thread") @pytest.mark.skipif(pytest_timeout==None, reason="Timeout package"\ " not installed; skipping test that may hang.") -@pytest.mark.timeout(10, method="thread") def test_remove_node_before_result(start_connected_cluster): """Removing a node should cause a Trial to be requeued.""" cluster = start_connected_cluster @@ -124,8 +124,8 @@ def test_remove_node_before_result(start_connected_cluster): cluster.remove_node(node) cluster.wait_for_nodes() - print("\n"*5) - print("!!!!!!!!!!!!!!!! Finished") + assert ray.global_state.cluster_resources["CPU"] == 1 + runner.step() # recover for i in range(5): runner.step() diff --git a/test/multi_node_test_2.py b/test/multi_node_test_2.py index f56ddd2e18b8..98f851d0073c 100644 --- a/test/multi_node_test_2.py +++ b/test/multi_node_test_2.py @@ -3,6 +3,7 @@ from __future__ import print_function import logging +import time import pytest import json From 3e96ec9b4a71814971b2f8583c3fab4117311c89 Mon Sep 17 00:00:00 2001 From: Robert Nishihara Date: Wed, 7 Nov 2018 09:14:19 -0800 Subject: [PATCH 22/74] Linting and Java fix --- .../src/main/java/org/ray/runtime/runner/RunManager.java | 1 + python/ray/scripts/scripts.py | 4 ++-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/java/runtime/src/main/java/org/ray/runtime/runner/RunManager.java b/java/runtime/src/main/java/org/ray/runtime/runner/RunManager.java index 8a5aab376b02..56940e33cbcf 100644 --- a/java/runtime/src/main/java/org/ray/runtime/runner/RunManager.java +++ b/java/runtime/src/main/java/org/ray/runtime/runner/RunManager.java @@ -187,6 +187,7 @@ private void startRaylet() { "0", // number of initial workers String.valueOf(maximumStartupConcurrency), ResourceUtil.getResourcesStringFromMap(rayConfig.resources), + "", // The internal config list. buildPythonWorkerCommand(), // python worker command buildWorkerCommandRaylet() // java worker command ); diff --git a/python/ray/scripts/scripts.py b/python/ray/scripts/scripts.py index b646c58eb11b..8f4e391fbea7 100644 --- a/python/ray/scripts/scripts.py +++ b/python/ray/scripts/scripts.py @@ -204,8 +204,8 @@ def start(node_ip_address, redis_address, redis_port, num_redis_shards, num_workers, num_cpus, num_gpus, resources, head, no_ui, block, plasma_directory, huge_pages, autoscaling_config, no_redirect_worker_output, no_redirect_output, - plasma_store_socket_name, raylet_socket_name, - temp_dir, internal_config): + plasma_store_socket_name, raylet_socket_name, temp_dir, + internal_config): # Convert hostnames to numerical IP address. if node_ip_address is not None: node_ip_address = services.address_to_ip(node_ip_address) From 4081c60e526d5e3075771e587e6d7aebe0a0e5b2 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 7 Nov 2018 14:47:20 -0800 Subject: [PATCH 23/74] add docstring --- test/multi_node_test_2.py | 42 ++++++++++++++++++++++++++------------- 1 file changed, 28 insertions(+), 14 deletions(-) diff --git a/test/multi_node_test_2.py b/test/multi_node_test_2.py index f56ddd2e18b8..339546be1359 100644 --- a/test/multi_node_test_2.py +++ b/test/multi_node_test_2.py @@ -2,9 +2,9 @@ from __future__ import division from __future__ import print_function +import json import logging import pytest -import json import ray import ray.services as services @@ -31,6 +31,24 @@ def start_connected_cluster(): g.shutdown() +@pytest.fixture +def start_connected_longer_cluster(): + """Creates a cluster with a longer timeout.""" + g = Cluster( + initialize_head=True, + connect=True, + head_node_args={ + "resources": dict(CPU=1), + "_internal_config": json.dumps({ + "num_heartbeats_timeout": 20 + }) + }) + yield g + # The code after the yield will run as teardown code. + ray.shutdown() + g.shutdown() + + def test_cluster(): """Basic test for adding and removing nodes in cluster.""" g = Cluster(initialize_head=False) @@ -43,16 +61,15 @@ def test_cluster(): assert not any(node.any_processes_alive() for node in g.list_all_nodes()) -def test_internal_config(): - cluster = Cluster( - initialize_head=True, - connect=True, - head_node_args={ - "resources": dict(CPU=1), - "_internal_config": json.dumps({ - "num_heartbeats_timeout": 20 - }) - }) +def test_internal_config(start_connected_longer_cluster): + """Checks that the internal configuration setting works. + + We set the cluster to timeout nodes after 2 seconds of no timeouts. We + then remove a node, wait for 1 second to check that the cluster is out + of sync, then wait another 2 seconds (giving 1 second of leeway) to check + that the client has timed out. + """ + cluster = start_connected_longer_cluster worker = cluster.add_node() cluster.wait_for_nodes() @@ -62,9 +79,6 @@ def test_internal_config(): cluster.wait_for_nodes(retries=20) assert ray.global_state.cluster_resources()["CPU"] == 1 - # The code after the yield will run as teardown code. - ray.shutdown() - cluster.shutdown() def test_wait_for_nodes(start_connected_cluster): From 2f5861c622ef270f9eb8d2871ae1ab087060d148 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 7 Nov 2018 22:43:21 -0800 Subject: [PATCH 24/74] Fix test, add assertions --- python/ray/test/cluster_utils.py | 3 +++ python/ray/test/test_global_state.py | 22 ++++++---------------- 2 files changed, 9 insertions(+), 16 deletions(-) diff --git a/python/ray/test/cluster_utils.py b/python/ray/test/cluster_utils.py index ee191c90cbf4..afaf5dd21921 100644 --- a/python/ray/test/cluster_utils.py +++ b/python/ray/test/cluster_utils.py @@ -118,6 +118,9 @@ def wait_for_nodes(self, retries=30): Args: retries (int): Number of times to retry checking client table. + + Returns: + True if successfully registered nodes as expected. """ for i in range(retries): diff --git a/python/ray/test/test_global_state.py b/python/ray/test/test_global_state.py index 2d64f627ec8c..e2003d9dc06f 100644 --- a/python/ray/test/test_global_state.py +++ b/python/ray/test/test_global_state.py @@ -2,17 +2,13 @@ from __future__ import division from __future__ import print_function +import json import pytest - try: import pytest_timeout except ModuleNotFoundError as e: pytest_timeout = None - import time -import os -import yaml -import tempfile import ray from ray.test.cluster_utils import Cluster @@ -30,19 +26,13 @@ def ray_start(): @pytest.fixture def cluster_start(): # Start the Ray processes. - memory_config = yaml.dump({"num_heartbeats_timeout": 10}) - buff = tempfile.NamedTemporaryFile(mode='w+', delete=False) - path = buff.name - buff.write(memory_config) - buff.close() cluster = Cluster( initialize_head=True, connect=True, head_node_args={ "resources": dict(CPU=1), - "config_file": path}) + "_internal_config": json.dumps( + {"num_heartbeats_timeout": 10})}) yield cluster - os.unlink(path) - # The code after the yield will run as teardown code. ray.shutdown() cluster.shutdown() @@ -98,14 +88,14 @@ def test_add_remove_cluster_resources(cluster_start): assert ray.global_state.cluster_resources()["CPU"] == 1 nodes = [] nodes += [cluster.add_node(resources=dict(CPU=1))] - cluster.wait_for_nodes() + assert cluster.wait_for_nodes() assert ray.global_state.cluster_resources()["CPU"] == 2 cluster.remove_node(nodes.pop()) - cluster.wait_for_nodes() + assert cluster.wait_for_nodes() assert ray.global_state.cluster_resources()["CPU"] == 1 for i in range(5): nodes += [cluster.add_node(resources=dict(CPU=1))] - cluster.wait_for_nodes() + assert cluster.wait_for_nodes() assert ray.global_state.cluster_resources()["CPU"] == 6 From 7f675f77e70faab268bd8ccaca1d41653aa3b374 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 7 Nov 2018 22:48:39 -0800 Subject: [PATCH 25/74] fix up tests --- python/ray/tune/test/cluster_tests.py | 87 ++++++++++++++------------- 1 file changed, 45 insertions(+), 42 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index d87402f26b3d..a320b8313cc9 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -32,7 +32,6 @@ def start_connected_cluster(): "_internal_config": json.dumps( {"num_heartbeats_timeout": 10})}) yield cluster - os.unlink(path) # The code after the yield will run as teardown code. ray.shutdown() cluster.shutdown() @@ -53,13 +52,16 @@ def _restore(self, state): self.state = state +@pytest.mark.skipif(pytest_timeout==None, reason="Timeout package"\ + " not installed; skipping test that may hang.") +@pytest.mark.timeout(10, method="thread") def test_counting_resources(start_connected_cluster): """Tests that Tune accounting is consistent with actual cluster.""" cluster = start_connected_cluster assert ray.global_state.cluster_resources()["CPU"] == 1 nodes = [] nodes += [cluster.add_node(resources=dict(CPU=1))] - cluster.wait_for_nodes() + assert cluster.wait_for_nodes() assert ray.global_state.cluster_resources()["CPU"] == 2 @@ -77,13 +79,13 @@ def test_counting_resources(start_connected_cluster): runner.step() # run 1 cluster.remove_node(nodes.pop()) - cluster.wait_for_nodes() + assert cluster.wait_for_nodes() assert ray.global_state.cluster_resources()["CPU"] == 1 runner.step() # run 2 for i in range(5): nodes += [cluster.add_node(resources=dict(CPU=1))] - cluster.wait_for_nodes() + assert cluster.wait_for_nodes() assert ray.global_state.cluster_resources()["CPU"] == 6 runner.step() # 1 result @@ -91,48 +93,49 @@ def test_counting_resources(start_connected_cluster): for i in range(5): node = nodes.pop() cluster.remove_node(node) - cluster.wait_for_nodes() + assert cluster.wait_for_nodes() assert ray.global_state.cluster_resources()["CPU"] == 1 +# Waiting for actor reconstruction PR +# @pytest.mark.skipif(pytest_timeout==None, reason="Timeout package"\ +# " not installed; skipping test that may hang.") # @pytest.mark.timeout(10, method="thread") -@pytest.mark.skipif(pytest_timeout==None, reason="Timeout package"\ - " not installed; skipping test that may hang.") -def test_remove_node_before_result(start_connected_cluster): - """Removing a node should cause a Trial to be requeued.""" - cluster = start_connected_cluster - node = cluster.add_node(resources=dict(CPU=1)) - - runner = TrialRunner(BasicVariantGenerator()) - kwargs = { - "stopping_criterion": { - "training_iteration": 3 - } - } - - tune.register_trainable("test", _Train) - trials = [Trial("test", **kwargs), Trial("test", **kwargs)] - for t in trials: - runner.add_trial(t) - - runner.step() # run 1 - runner.step() # run 2 - assert all(t.status == Trial.RUNNING for t in trials) - - runner.step() # 1 result - print(runner.debug_string()) - - cluster.remove_node(node) - cluster.wait_for_nodes() - assert ray.global_state.cluster_resources["CPU"] == 1 - - runner.step() # recover - for i in range(5): - runner.step() - assert all(t.status == Trial.TERMINATED for t in trials) - - with pytest.raises(TuneError): - runner.step() +# def test_remove_node_before_result(start_connected_cluster): +# """Removing a node should cause a Trial to be requeued.""" +# cluster = start_connected_cluster +# node = cluster.add_node(resources=dict(CPU=1)) + +# runner = TrialRunner(BasicVariantGenerator()) +# kwargs = { +# "stopping_criterion": { +# "training_iteration": 3 +# } +# } + +# tune.register_trainable("test", _Train) +# trials = [Trial("test", **kwargs), Trial("test", **kwargs)] +# for t in trials: +# runner.add_trial(t) + +# runner.step() # run 1 +# runner.step() # run 2 +# assert all(t.status == Trial.RUNNING for t in trials) + +# runner.step() # 1 result +# print(runner.debug_string()) + +# cluster.remove_node(node) +# cluster.wait_for_nodes() +# assert ray.global_state.cluster_resources["CPU"] == 1 + +# runner.step() # recover +# for i in range(5): +# runner.step() +# assert all(t.status == Trial.TERMINATED for t in trials) + +# with pytest.raises(TuneError): +# runner.step() def test_trial_migration(start_connected_cluster): From af0fe9cc9e332c33244280dd917c33183ec3b513 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 7 Nov 2018 22:52:26 -0800 Subject: [PATCH 26/74] pytest ext --- .travis/install-dependencies.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/.travis/install-dependencies.sh b/.travis/install-dependencies.sh index d3662c157207..293c1b8b6b04 100755 --- a/.travis/install-dependencies.sh +++ b/.travis/install-dependencies.sh @@ -25,7 +25,7 @@ if [[ "$PYTHON" == "2.7" ]] && [[ "$platform" == "linux" ]]; then bash miniconda.sh -b -p $HOME/miniconda export PATH="$HOME/miniconda/bin:$PATH" pip install -q cython==0.27.3 cmake tensorflow gym opencv-python pyyaml pandas==0.22 requests \ - feather-format lxml openpyxl xlrd py-spy setproctitle faulthandler + feather-format lxml openpyxl xlrd py-spy setproctitle faulthandler pytest-timeout elif [[ "$PYTHON" == "3.5" ]] && [[ "$platform" == "linux" ]]; then sudo apt-get update sudo apt-get install -y cmake pkg-config python-dev python-numpy build-essential autoconf curl libtool unzip @@ -34,7 +34,7 @@ elif [[ "$PYTHON" == "3.5" ]] && [[ "$platform" == "linux" ]]; then bash miniconda.sh -b -p $HOME/miniconda export PATH="$HOME/miniconda/bin:$PATH" pip install -q cython==0.27.3 cmake tensorflow gym opencv-python pyyaml pandas==0.22 requests \ - feather-format lxml openpyxl xlrd py-spy setproctitle + feather-format lxml openpyxl xlrd py-spy setproctitle pytest-timeout elif [[ "$PYTHON" == "2.7" ]] && [[ "$platform" == "macosx" ]]; then # check that brew is installed which -s brew @@ -51,7 +51,7 @@ elif [[ "$PYTHON" == "2.7" ]] && [[ "$platform" == "macosx" ]]; then bash miniconda.sh -b -p $HOME/miniconda export PATH="$HOME/miniconda/bin:$PATH" pip install -q cython==0.27.3 cmake tensorflow gym opencv-python pyyaml pandas==0.22 requests \ - feather-format lxml openpyxl xlrd py-spy setproctitle faulthandler + feather-format lxml openpyxl xlrd py-spy setproctitle faulthandler pytest-timeout elif [[ "$PYTHON" == "3.5" ]] && [[ "$platform" == "macosx" ]]; then # check that brew is installed which -s brew @@ -68,7 +68,7 @@ elif [[ "$PYTHON" == "3.5" ]] && [[ "$platform" == "macosx" ]]; then bash miniconda.sh -b -p $HOME/miniconda export PATH="$HOME/miniconda/bin:$PATH" pip install -q cython==0.27.3 cmake tensorflow gym opencv-python pyyaml pandas==0.22 requests \ - feather-format lxml openpyxl xlrd py-spy setproctitle + feather-format lxml openpyxl xlrd py-spy setproctitle pytest-timeout elif [[ "$LINT" == "1" ]]; then sudo apt-get update sudo apt-get install -y cmake build-essential autoconf curl libtool unzip From d01d80c5ffd35bb27779e77fcce4b0815c524692 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 12 Nov 2018 01:11:32 -0800 Subject: [PATCH 27/74] code to make requeueing work --- python/ray/tune/test/cluster_tests.py | 136 ++++++++++++++++++++------ python/ray/tune/trial.py | 8 ++ python/ray/tune/trial_executor.py | 22 ++--- python/ray/tune/trial_runner.py | 2 +- 4 files changed, 126 insertions(+), 42 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index a320b8313cc9..d96bc4725664 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -5,6 +5,7 @@ import os import json import tempfile +import time import pytest try: import pytest_timeout @@ -21,6 +22,22 @@ from ray.tune.suggest import grid_search, BasicVariantGenerator +class _Train(tune.Trainable): + def _setup(self, config): + self.state = {"hi": 1} + + def _train(self): + self.state["hi"] += 1 + time.sleep(0.5) + return {} + + def _save(self, path): + return self.state + + def _restore(self, state): + self.state = state + + @pytest.fixture def start_connected_cluster(): # Start the Ray processes. @@ -37,19 +54,21 @@ def start_connected_cluster(): cluster.shutdown() -class _Train(tune.Trainable): - def _setup(self, config): - self.state = {"hi": 1} - - def _train(self): - self.state["hi"] += 1 - return {} - - def _save(self, path): - return self.state +@pytest.fixture +def start_connected_emptyhead_cluster(): + """Starts head with no resources.""" - def _restore(self, state): - self.state = state + cluster = Cluster( + initialize_head=True, connect=True, + head_node_args={ + "resources": dict(CPU=0), + "_internal_config": json.dumps( + {"num_heartbeats_timeout": 10})}) + tune.register_trainable("test", _Train) + yield cluster + # The code after the yield will run as teardown code. + ray.shutdown() + cluster.shutdown() @pytest.mark.skipif(pytest_timeout==None, reason="Timeout package"\ @@ -97,7 +116,6 @@ def test_counting_resources(start_connected_cluster): assert ray.global_state.cluster_resources()["CPU"] == 1 -# Waiting for actor reconstruction PR # @pytest.mark.skipif(pytest_timeout==None, reason="Timeout package"\ # " not installed; skipping test that may hang.") # @pytest.mark.timeout(10, method="thread") @@ -138,16 +156,84 @@ def test_counting_resources(start_connected_cluster): # runner.step() -def test_trial_migration(start_connected_cluster): - """Removing a node should cause a Trial to be requeued.""" - cluster = start_connected_cluster +def test_trial_migration(start_connected_emptyhead_cluster): + """Removing a node while cluster has space should migrate trial. + + The trial state should also be consistent with the checkpoint. + """ + cluster = start_connected_emptyhead_cluster node = cluster.add_node(resources=dict(CPU=1)) + assert cluster.wait_for_nodes() runner = TrialRunner(BasicVariantGenerator()) kwargs = { "stopping_criterion": { "training_iteration": 3 - } + }, + "checkpoint_freq": 2, + "max_failures": 2 + } + + # Test recovery of trial that hasn't been checkpointed + t = Trial("test", **kwargs) + runner.add_trial(t) + runner.step() # start + runner.step() # 1 result + node2 = cluster.add_node(resources=dict(CPU=1)) + cluster.remove_node(node) + assert cluster.wait_for_nodes() + runner.step() # Recovery step + for i in range(3): + runner.step() + + assert t.status == Trial.TERMINATED + + # Test recovery of trial that has been checkpointed + t2 = Trial("test", **kwargs) + runner.add_trial(t2) + runner.step() # start + runner.step() # 1 result + runner.step() # 2 result and checkpoint + assert t2.has_checkpoint() + node3 = cluster.add_node(resources=dict(CPU=1)) + cluster.remove_node(node2) + assert cluster.wait_for_nodes() + runner.step() # Recovery step + for i in range(1): + runner.step() + + assert t2.status == Trial.TERMINATED + + # Test recovery of trial that won't be checkpointed + t3 = Trial("test", **{ + "stopping_criterion": { + "training_iteration": 3 + }}) + runner.add_trial(t3) + runner.step() # start + runner.step() # 1 result + node4 = cluster.add_node(resources=dict(CPU=1)) + cluster.remove_node(node3) + assert cluster.wait_for_nodes() + runner.step() # Error handling step + assert t3.status == Trial.ERROR + + with pytest.raises(TuneError): + runner.step() + + +def test_trial_requeue(start_connected_emptyhead_cluster): + """Removing a node in full cluster causes Trial to be requeued.""" + cluster = start_connected_emptyhead_cluster + node = cluster.add_node(resources=dict(CPU=1)) + + runner = TrialRunner(BasicVariantGenerator()) + kwargs = { + "stopping_criterion": { + "training_iteration": 5 + }, + "checkpoint_freq": 1, + "max_failures": 1 } tune.register_trainable("test", _Train) @@ -155,22 +241,14 @@ def test_trial_migration(start_connected_cluster): for t in trials: runner.add_trial(t) - runner.step() # run 1 - runner.step() # run 2 - assert all(t.status == Trial.RUNNING for t in trials) - + runner.step() # start runner.step() # 1 result print(runner.debug_string()) cluster.remove_node(node) - cluster.wait_for_nodes() - node2 = cluster.add_node(resources=dict(CPU=1)) - - runner.step() # recover - for i in range(5): - runner.step() - print(runner.debug_string()) - assert all(t.status == Trial.TERMINATED for t in trials) + assert cluster.wait_for_nodes() + runner.step() + assert all(t.status == Trial.PENDING for t in trials) with pytest.raises(TuneError): runner.step() diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index 59559ebbe2c2..38cc1dee1cac 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -277,6 +277,14 @@ def _status_string(self): def has_checkpoint(self): return self._checkpoint.value is not None + def should_recover(self): + if self.last_result is None: + return True + elif (self.checkpoint_freq > 0 and + self.checkpoint_freq > self.last_result[TRAINING_ITERATION]): + return True + return self.has_checkpoint() + def update_last_result(self, result, terminate=False): if terminate: result.update(done=True) diff --git a/python/ray/tune/trial_executor.py b/python/ray/tune/trial_executor.py index b961d12b8cbe..bfbd8f697fc5 100644 --- a/python/ray/tune/trial_executor.py +++ b/python/ray/tune/trial_executor.py @@ -61,24 +61,22 @@ def stop_trial(self, trial, error=False, error_msg=None, stop_logger=True): "stop_trial() method") def restart_trial(self, trial, error_msg=None): - """Restarts the trial. + """Restarts or requeues the trial. - The state of the trial should restore from the last checkpoint. + The state of the trial should restore from the last checkpoint. Trial + is requeued if the cluster no longer has resources to accomodate it. Args: error_msg (str): Optional error message. """ - try: - logger.info( - "Attempting to recover trial state from last checkpoint") - self.stop_trial( - trial, error=True, error_msg=error_msg, stop_logger=False) - trial.result_logger.flush() + self.stop_trial( + trial, error=error_msg is not None, + error_msg=error_msg, stop_logger=False) + trial.result_logger.flush() + if self.has_resources(trial.resources): self.start_trial(trial) - except Exception: - error_msg = traceback.format_exc() - logger.exception("Error recovering trial from checkpoint, abort.") - self.stop_trial(trial, error=True, error_msg=error_msg) + else: + trial.status = Trial.PENDING def continue_training(self, trial): """Continues the training of this trial.""" diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index bb31345c8f5c..dc08b07915c6 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -277,7 +277,7 @@ def _process_events(self): logger.exception("Error processing event.") error_msg = traceback.format_exc() if trial.status == Trial.RUNNING: - if trial.has_checkpoint() and \ + if trial.should_recover() and \ trial.num_failures < trial.max_failures: self._try_recover(trial, error_msg) else: From 1e322278ccaf52a5e3de34f09b0080956b6de3c9 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 12 Nov 2018 01:11:57 -0800 Subject: [PATCH 28/74] yapf --- python/ray/tune/test/cluster_tests.py | 30 +++++++++++++-------------- 1 file changed, 14 insertions(+), 16 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index d96bc4725664..45b0c2fca8c4 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -43,11 +43,14 @@ def start_connected_cluster(): # Start the Ray processes. cluster = Cluster( - initialize_head=True, connect=True, + initialize_head=True, + connect=True, head_node_args={ "resources": dict(CPU=1), - "_internal_config": json.dumps( - {"num_heartbeats_timeout": 10})}) + "_internal_config": json.dumps({ + "num_heartbeats_timeout": 10 + }) + }) yield cluster # The code after the yield will run as teardown code. ray.shutdown() @@ -59,11 +62,14 @@ def start_connected_emptyhead_cluster(): """Starts head with no resources.""" cluster = Cluster( - initialize_head=True, connect=True, + initialize_head=True, + connect=True, head_node_args={ "resources": dict(CPU=0), - "_internal_config": json.dumps( - {"num_heartbeats_timeout": 10})}) + "_internal_config": json.dumps({ + "num_heartbeats_timeout": 10 + }) + }) tune.register_trainable("test", _Train) yield cluster # The code after the yield will run as teardown code. @@ -83,13 +89,8 @@ def test_counting_resources(start_connected_cluster): assert cluster.wait_for_nodes() assert ray.global_state.cluster_resources()["CPU"] == 2 - runner = TrialRunner(BasicVariantGenerator()) - kwargs = { - "stopping_criterion": { - "training_iteration": 10 - } - } + kwargs = {"stopping_criterion": {"training_iteration": 10}} tune.register_trainable("test", _Train) trials = [Trial("test", **kwargs), Trial("test", **kwargs)] @@ -205,10 +206,7 @@ def test_trial_migration(start_connected_emptyhead_cluster): assert t2.status == Trial.TERMINATED # Test recovery of trial that won't be checkpointed - t3 = Trial("test", **{ - "stopping_criterion": { - "training_iteration": 3 - }}) + t3 = Trial("test", **{"stopping_criterion": {"training_iteration": 3}}) runner.add_trial(t3) runner.step() # start runner.step() # 1 result From 9db9d1660e1c768f4697873b8cd7d353d7c73c4a Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 12 Nov 2018 01:14:27 -0800 Subject: [PATCH 29/74] lint --- python/ray/tune/test/cluster_tests.py | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 45b0c2fca8c4..15e76a3c33ea 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -2,9 +2,7 @@ from __future__ import division from __future__ import print_function -import os import json -import tempfile import time import pytest try: @@ -16,10 +14,9 @@ import ray from ray import tune from ray.tune.error import TuneError -from ray.tune.experiment import Experiment -from ray.tune.trial import Trial, Resources +from ray.tune.trial import Trial from ray.tune.trial_runner import TrialRunner -from ray.tune.suggest import grid_search, BasicVariantGenerator +from ray.tune.suggest import BasicVariantGenerator class _Train(tune.Trainable): @@ -77,11 +74,12 @@ def start_connected_emptyhead_cluster(): cluster.shutdown() -@pytest.mark.skipif(pytest_timeout==None, reason="Timeout package"\ - " not installed; skipping test that may hang.") +@pytest.mark.skipif(pytest_timeout is None, reason="Timeout package" + " not installed; skipping test that may hang.") @pytest.mark.timeout(10, method="thread") def test_counting_resources(start_connected_cluster): """Tests that Tune accounting is consistent with actual cluster.""" + cluster = start_connected_cluster assert ray.global_state.cluster_resources()["CPU"] == 1 nodes = [] @@ -210,7 +208,7 @@ def test_trial_migration(start_connected_emptyhead_cluster): runner.add_trial(t3) runner.step() # start runner.step() # 1 result - node4 = cluster.add_node(resources=dict(CPU=1)) + cluster.add_node(resources=dict(CPU=1)) cluster.remove_node(node3) assert cluster.wait_for_nodes() runner.step() # Error handling step From 2639a98e6c1b29867b5da5df33e71d666a9a0cb3 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 12 Nov 2018 01:22:11 -0800 Subject: [PATCH 30/74] comments --- python/ray/tune/test/cluster_tests.py | 1 + python/ray/tune/trial.py | 13 +++++++++---- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 15e76a3c33ea..af9f0b119f97 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -115,6 +115,7 @@ def test_counting_resources(start_connected_cluster): assert ray.global_state.cluster_resources()["CPU"] == 1 +# TODO: Emable this after reconstruction is in. # @pytest.mark.skipif(pytest_timeout==None, reason="Timeout package"\ # " not installed; skipping test that may hang.") # @pytest.mark.timeout(10, method="thread") diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index 38cc1dee1cac..c0de5c871b93 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -278,10 +278,15 @@ def has_checkpoint(self): return self._checkpoint.value is not None def should_recover(self): - if self.last_result is None: - return True - elif (self.checkpoint_freq > 0 and - self.checkpoint_freq > self.last_result[TRAINING_ITERATION]): + """Returns whether the trial qualifies for restoring. + + This is if a checkpoint frequency is set, or if there exists + a pre-existing checkpoint. + """ + if self.checkpoint_freq > 0: + # Edge case of begin ning trial + if (self.last_result is None or + self.checkpoint_freq > self.last_result[TRAINING_ITERATION]): return True return self.has_checkpoint() From d45c74b992a93e1579bdbd5022d071ede6c69f07 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 12 Nov 2018 01:22:55 -0800 Subject: [PATCH 31/74] lint --- python/ray/tune/trial.py | 6 +++--- python/ray/tune/trial_executor.py | 6 ++++-- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index c0de5c871b93..4ff43cd3f839 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -285,9 +285,9 @@ def should_recover(self): """ if self.checkpoint_freq > 0: # Edge case of begin ning trial - if (self.last_result is None or - self.checkpoint_freq > self.last_result[TRAINING_ITERATION]): - return True + if (self.last_result is None or self.checkpoint_freq > + self.last_result[TRAINING_ITERATION]): + return True return self.has_checkpoint() def update_last_result(self, result, terminate=False): diff --git a/python/ray/tune/trial_executor.py b/python/ray/tune/trial_executor.py index bfbd8f697fc5..24f65f6f36e6 100644 --- a/python/ray/tune/trial_executor.py +++ b/python/ray/tune/trial_executor.py @@ -70,8 +70,10 @@ def restart_trial(self, trial, error_msg=None): error_msg (str): Optional error message. """ self.stop_trial( - trial, error=error_msg is not None, - error_msg=error_msg, stop_logger=False) + trial, + error=error_msg is not None, + error_msg=error_msg, + stop_logger=False) trial.result_logger.flush() if self.has_resources(trial.resources): self.start_trial(trial) From ee6a800ce056e2169cb18f011da844331694f897 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 12 Nov 2018 01:24:23 -0800 Subject: [PATCH 32/74] Update multi_node_test_2.py --- test/multi_node_test_2.py | 1 - 1 file changed, 1 deletion(-) diff --git a/test/multi_node_test_2.py b/test/multi_node_test_2.py index 1282dce61e80..339546be1359 100644 --- a/test/multi_node_test_2.py +++ b/test/multi_node_test_2.py @@ -4,7 +4,6 @@ import json import logging -import time import pytest import ray From 3f13bfa2ba391050c5a915839904b77b5ef2eb87 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 12 Nov 2018 01:24:41 -0800 Subject: [PATCH 33/74] lit --- python/ray/tune/trial.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index 4ff43cd3f839..0ccd9074f399 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -284,10 +284,10 @@ def should_recover(self): a pre-existing checkpoint. """ if self.checkpoint_freq > 0: - # Edge case of begin ning trial - if (self.last_result is None or self.checkpoint_freq > - self.last_result[TRAINING_ITERATION]): - return True + # Edge case of beginning trial + if (self.last_result is None or + self.checkpoint_freq > self.last_result[TRAINING_ITERATION]): + return True return self.has_checkpoint() def update_last_result(self, result, terminate=False): From 5f0d75e84536344ef8827244e17ec6c6a95c7819 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 12 Nov 2018 11:49:50 -0800 Subject: [PATCH 34/74] re-enable --- python/ray/tune/test/cluster_tests.py | 78 +++++++++++++-------------- python/ray/tune/trial.py | 6 +-- python/ray/tune/trial_executor.py | 1 - 3 files changed, 42 insertions(+), 43 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index af9f0b119f97..ab9e1786dc5f 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -115,45 +115,45 @@ def test_counting_resources(start_connected_cluster): assert ray.global_state.cluster_resources()["CPU"] == 1 -# TODO: Emable this after reconstruction is in. -# @pytest.mark.skipif(pytest_timeout==None, reason="Timeout package"\ -# " not installed; skipping test that may hang.") -# @pytest.mark.timeout(10, method="thread") -# def test_remove_node_before_result(start_connected_cluster): -# """Removing a node should cause a Trial to be requeued.""" -# cluster = start_connected_cluster -# node = cluster.add_node(resources=dict(CPU=1)) - -# runner = TrialRunner(BasicVariantGenerator()) -# kwargs = { -# "stopping_criterion": { -# "training_iteration": 3 -# } -# } - -# tune.register_trainable("test", _Train) -# trials = [Trial("test", **kwargs), Trial("test", **kwargs)] -# for t in trials: -# runner.add_trial(t) - -# runner.step() # run 1 -# runner.step() # run 2 -# assert all(t.status == Trial.RUNNING for t in trials) - -# runner.step() # 1 result -# print(runner.debug_string()) - -# cluster.remove_node(node) -# cluster.wait_for_nodes() -# assert ray.global_state.cluster_resources["CPU"] == 1 - -# runner.step() # recover -# for i in range(5): -# runner.step() -# assert all(t.status == Trial.TERMINATED for t in trials) - -# with pytest.raises(TuneError): -# runner.step() +@pytest.mark.skip("Add this test once reconstruction is fixed") +@pytest.mark.skipif(pytest_timeout is None, reason="Timeout package" + " not installed; skipping test that may hang.") +@pytest.mark.timeout(10, method="thread") +def test_remove_node_before_result(start_connected_cluster): + """Removing a node should cause a Trial to be requeued.""" + cluster = start_connected_cluster + node = cluster.add_node(resources=dict(CPU=1)) + + runner = TrialRunner(BasicVariantGenerator()) + kwargs = { + "stopping_criterion": { + "training_iteration": 3 + } + } + + tune.register_trainable("test", _Train) + trials = [Trial("test", **kwargs), Trial("test", **kwargs)] + for t in trials: + runner.add_trial(t) + + runner.step() # run 1 + runner.step() # run 2 + assert all(t.status == Trial.RUNNING for t in trials) + + runner.step() # 1 result + print(runner.debug_string()) + + cluster.remove_node(node) + cluster.wait_for_nodes() + assert ray.global_state.cluster_resources["CPU"] == 1 + + runner.step() # recover + for i in range(5): + runner.step() + assert all(t.status == Trial.TERMINATED for t in trials) + + with pytest.raises(TuneError): + runner.step() def test_trial_migration(start_connected_emptyhead_cluster): diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index 0ccd9074f399..56b3bc230862 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -285,9 +285,9 @@ def should_recover(self): """ if self.checkpoint_freq > 0: # Edge case of beginning trial - if (self.last_result is None or - self.checkpoint_freq > self.last_result[TRAINING_ITERATION]): - return True + if (self.checkpoint_freq > self.last_result[TRAINING_ITERATION] + or self.last_result is None): + return True return self.has_checkpoint() def update_last_result(self, result, terminate=False): diff --git a/python/ray/tune/trial_executor.py b/python/ray/tune/trial_executor.py index 24f65f6f36e6..bd36a84761e5 100644 --- a/python/ray/tune/trial_executor.py +++ b/python/ray/tune/trial_executor.py @@ -4,7 +4,6 @@ from __future__ import print_function import logging -import traceback from ray.tune.trial import Trial, Checkpoint From b1793bd6b5aa44c763e4ba64ed9f02e80aff0016 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 12 Nov 2018 11:53:14 -0800 Subject: [PATCH 35/74] lint --- python/ray/test/test_global_state.py | 25 ++++++++++++++++--------- 1 file changed, 16 insertions(+), 9 deletions(-) diff --git a/python/ray/test/test_global_state.py b/python/ray/test/test_global_state.py index e2003d9dc06f..8dc288736b5f 100644 --- a/python/ray/test/test_global_state.py +++ b/python/ray/test/test_global_state.py @@ -27,19 +27,23 @@ def ray_start(): def cluster_start(): # Start the Ray processes. cluster = Cluster( - initialize_head=True, connect=True, + initialize_head=True, + connect=True, head_node_args={ "resources": dict(CPU=1), - "_internal_config": json.dumps( - {"num_heartbeats_timeout": 10})}) + "_internal_config": json.dumps({ + "num_heartbeats_timeout": 10 + }) + }) yield cluster ray.shutdown() cluster.shutdown() # TODO(rliaw): The proper way to do this is to have the pytest config setup. -@pytest.mark.skipif(pytest_timeout==None, reason="Timeout package"\ - " not installed; skipping test that may hang.") +@pytest.mark.skipif( + pytest_timeout is None, + reason="Timeout package not installed; skipping test that may hang.") @pytest.mark.timeout(10) def test_replenish_resources(ray_start): cluster_resources = ray.global_state.cluster_resources() @@ -59,11 +63,13 @@ def cpu_task(): assert resources_reset -@pytest.mark.skipif(pytest_timeout==None, reason="Timeout package"\ - " not installed; skipping test that may hang.") +@pytest.mark.skipif( + pytest_timeout is None, + reason="Timeout package not installed; skipping test that may hang.") @pytest.mark.timeout(10) def test_uses_resources(ray_start): cluster_resources = ray.global_state.cluster_resources() + @ray.remote def cpu_task(): time.sleep(1) @@ -79,8 +85,9 @@ def cpu_task(): assert resource_used -@pytest.mark.skipif(pytest_timeout==None, reason="Timeout package"\ - " not installed; skipping test that may hang.") +@pytest.mark.skipif( + pytest_timeout is None, + reason="Timeout package not installed; skipping test that may hang.") @pytest.mark.timeout(20) def test_add_remove_cluster_resources(cluster_start): """Tests that Global State API is consistent with actual cluster.""" From 07630043273234d6865991d220e043bfa360eb52 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 12 Nov 2018 13:43:18 -0800 Subject: [PATCH 36/74] initial nuke test --- python/ray/tune/test/cluster_tests.py | 28 +++++++++++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index ab9e1786dc5f..a17795e5cdf9 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -249,3 +249,31 @@ def test_trial_requeue(start_connected_emptyhead_cluster): with pytest.raises(TuneError): runner.step() + + +def test_cluster_down(start_connected_emptyhead_cluster): + """Removing a node in full cluster causes Trial to be requeued.""" + cluster = start_connected_emptyhead_cluster + node = cluster.add_node(resources=dict(CPU=1)) + + runner = TrialRunner(BasicVariantGenerator()) + kwargs = { + "stopping_criterion": { + "training_iteration": 5 + }, + "checkpoint_freq": 1, + "max_failures": 1 + } + + tune.register_trainable("test", _Train) + trials = [Trial("test", **kwargs), Trial("test", **kwargs)] + for t in trials: + runner.add_trial(t) + + runner.step() # start + runner.step() # 1 result + checkpoint = runner.checkpoint() + cluster.shutdown() + _start_new_cluster() + runner = TrialRunner.from_checkpoint(checkpoint) + runner.step() From a1a05f0eaaf63ad5950fb5d69cc7c4da6f06ede7 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 12 Nov 2018 14:16:57 -0800 Subject: [PATCH 37/74] Track last result --- python/ray/tune/ray_trial_executor.py | 3 +++ python/ray/tune/test/cluster_tests.py | 7 +++++++ python/ray/tune/trial.py | 3 ++- 3 files changed, 12 insertions(+), 1 deletion(-) diff --git a/python/ray/tune/ray_trial_executor.py b/python/ray/tune/ray_trial_executor.py index 4a216a60d2be..9b152492ca80 100644 --- a/python/ray/tune/ray_trial_executor.py +++ b/python/ray/tune/ray_trial_executor.py @@ -267,6 +267,7 @@ def on_step_begin(self): def save(self, trial, storage=Checkpoint.DISK): """Saves the trial's state to a checkpoint.""" trial._checkpoint.storage = storage + trial._checkpoint.last_result = trial.last_result if storage == Checkpoint.MEMORY: trial._checkpoint.value = trial.runner.save_to_object.remote() else: @@ -290,6 +291,8 @@ def restore(self, trial, checkpoint=None): ray.get(trial.runner.restore_from_object.remote(value)) else: ray.get(trial.runner.restore.remote(value)) + trial.last_result = checkpoint.last_result + return True except Exception: logger.exception("Error restoring runner.") diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index ab9e1786dc5f..07a62eccf1f5 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -179,10 +179,16 @@ def test_trial_migration(start_connected_emptyhead_cluster): runner.add_trial(t) runner.step() # start runner.step() # 1 result + assert t.last_result is not None node2 = cluster.add_node(resources=dict(CPU=1)) cluster.remove_node(node) assert cluster.wait_for_nodes() runner.step() # Recovery step + + # TODO(rliaw): This assertion is not critical but will not pass + # because checkpoint handling is clean and should be refactored + # rather than hotfixed. + # assert t.last_result is None, "Trial result not restored correctly." for i in range(3): runner.step() @@ -199,6 +205,7 @@ def test_trial_migration(start_connected_emptyhead_cluster): cluster.remove_node(node2) assert cluster.wait_for_nodes() runner.step() # Recovery step + assert t2.last_result["training_iteration"] == 2 for i in range(1): runner.step() diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index 56b3bc230862..8e0526c7aab8 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -85,9 +85,10 @@ class Checkpoint(object): MEMORY = "memory" DISK = "disk" - def __init__(self, storage, value): + def __init__(self, storage, value, last_result=None): self.storage = storage self.value = value + self.last_result = last_result @staticmethod def from_object(value=None): From 6ecc2bb07f46b2b9e5e4a339dfd5233517c5ad40 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 12 Nov 2018 14:33:35 -0800 Subject: [PATCH 38/74] note --- python/ray/tune/test/cluster_tests.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 07a62eccf1f5..4697ac69b443 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -186,7 +186,7 @@ def test_trial_migration(start_connected_emptyhead_cluster): runner.step() # Recovery step # TODO(rliaw): This assertion is not critical but will not pass - # because checkpoint handling is clean and should be refactored + # because checkpoint handling is messy and should be refactored # rather than hotfixed. # assert t.last_result is None, "Trial result not restored correctly." for i in range(3): From 09995fa7684a448cef97f60ab825c679a112d30e Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 12 Nov 2018 14:45:42 -0800 Subject: [PATCH 39/74] add checkpointing to trial_runner --- python/ray/tune/trial_runner.py | 46 +++++++++++++++++++++++++++++---- 1 file changed, 41 insertions(+), 5 deletions(-) diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index dc08b07915c6..d94a136e96c9 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -8,6 +8,7 @@ import re import time import traceback +import pickle from ray.tune import TuneError from ray.tune.ray_trial_executor import RayTrialExecutor @@ -90,6 +91,38 @@ def __init__(self, self._stop_queue = [] self._verbose = verbose self._queue_trials = queue_trials + self._checkpoints = {} + + + def save(self, checkpoint_dir, force=False): + # search_alg_checkpoint = self._search_alg.save(checkpoint_dir) + # scheduler_alg_checkpoint = self._scheduler_alg.save(checkpoint_dir) + runner_state = { + "checkpoints": pickle.dumps(list(self._checkpoints.values())), + "total_time": self._total_time, + "stop_queue": self._stop_queue + } + with open(os.path.join(checkpoint_dir, "experiment.state"), "wb") as f: + pickle.dump(runner_state, f) + + def restore(self, checkpoint_dir): + logger.debug("Stopping all trials.") + for trial in self._trials: + self.stop_trial(trial) + + with open(os.path.join(checkpoint_dir, "experiment.state"), "rb") as f: + state = pickle.load(f) + + logger.info("Replacing all trials with checkpoint state.") + runner_state = state[0] + checkpoints = pickle.loads(runner_state["checkpoints"]) + + for ckpt in checkpoints: + self.add_trial(self.trial_executor.recreate_from_checkpoint(ckpt)) + + self._total_time = runner_state["total_time"] + self._stop_queue = runner_state["stop_queue"] + def is_finished(self): """Returns whether all trials have finished running.""" @@ -258,17 +291,14 @@ def _process_events(self): result, terminate=(decision == TrialScheduler.STOP)) if decision == TrialScheduler.CONTINUE: - if trial.should_checkpoint(result): - # TODO(rliaw): This is a blocking call - self.trial_executor.save(trial) + self._checkpoint_if_needed(trial, result) self.trial_executor.continue_training(trial) elif decision == TrialScheduler.PAUSE: self.trial_executor.pause_trial(trial) elif decision == TrialScheduler.STOP: # Checkpoint before ending the trial # if checkpoint_at_end experiment option is set to True - if trial.should_checkpoint(result): - self.trial_executor.save(trial) + self._checkpoint_if_needed(trial, result) self.trial_executor.stop_trial(trial) else: assert False, "Invalid scheduling decision: {}".format( @@ -286,6 +316,12 @@ def _process_events(self): trial.trial_id, error=True) self.trial_executor.stop_trial(trial, True, error_msg) + def _checkpoint_if_needed(self, trial, result): + if trial.should_checkpoint(result): + # TODO(rliaw): This is a blocking call + checkpoint = self.trial_executor.save_trial(trial) + self._checkpoints[trial] = checkpoint + def _try_recover(self, trial, error_msg): try: logger.info("Attempting to recover" From 24d6e12457a04a0b642ea1a1bf4b078dc259ba26 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 12 Nov 2018 14:51:19 -0800 Subject: [PATCH 40/74] trialrunners --- python/ray/tune/trial_runner.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index d94a136e96c9..e80b11ac912e 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -93,8 +93,7 @@ def __init__(self, self._queue_trials = queue_trials self._checkpoints = {} - - def save(self, checkpoint_dir, force=False): + def save(self, checkpoint_dir): # search_alg_checkpoint = self._search_alg.save(checkpoint_dir) # scheduler_alg_checkpoint = self._scheduler_alg.save(checkpoint_dir) runner_state = { @@ -123,7 +122,6 @@ def restore(self, checkpoint_dir): self._total_time = runner_state["total_time"] self._stop_queue = runner_state["stop_queue"] - def is_finished(self): """Returns whether all trials have finished running.""" From dbd1bbc62f6401941d4de24435f695253bd461aa Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 12 Nov 2018 14:55:38 -0800 Subject: [PATCH 41/74] logging --- python/ray/tune/logger.py | 31 +++++++++++++++++++++---------- 1 file changed, 21 insertions(+), 10 deletions(-) diff --git a/python/ray/tune/logger.py b/python/ray/tune/logger.py index f73aa4a1ef8c..87f7e026d892 100644 --- a/python/ray/tune/logger.py +++ b/python/ray/tune/logger.py @@ -71,19 +71,19 @@ def _init(self): self._log_syncer = get_syncer(self.logdir, self.uri) def on_result(self, result): - for logger in self._loggers: - logger.on_result(result) + for _logger in self._loggers: + _logger.on_result(result) self._log_syncer.set_worker_ip(result.get(NODE_IP)) self._log_syncer.sync_if_needed() def close(self): - for logger in self._loggers: - logger.close() + for _logger in self._loggers: + _logger.close() self._log_syncer.sync_now(force=True) def flush(self): - for logger in self._loggers: - logger.flush() + for _logger in self._loggers: + _logger.flush() self._log_syncer.sync_now(force=True) self._log_syncer.wait() @@ -99,7 +99,7 @@ def _init(self): with open(config_out, "w") as f: json.dump(self.config, f, sort_keys=True, cls=_SafeFallbackEncoder) local_file = os.path.join(self.logdir, "result.json") - self.local_out = open(local_file, "w") + self.local_out = open(local_file, "a") def on_result(self, result): json.dump(result, self, cls=_SafeFallbackEncoder) @@ -109,6 +109,9 @@ def write(self, b): self.local_out.write(b) self.local_out.flush() + def flush(self): + self.local_out.flush() + def close(self): self.local_out.close() @@ -128,6 +131,7 @@ def to_tf_values(result, path): class _TFLogger(Logger): def _init(self): + # TODO(rliaw): Implement a proper resume functionality for this. self._file_writer = tf.summary.FileWriter(self.logdir) def on_result(self, result): @@ -135,7 +139,8 @@ def on_result(self, result): for k in [ "config", "pid", "timestamp", TIME_TOTAL_S, TRAINING_ITERATION ]: - del tmp[k] # not useful to tf log these + if k in tmp: + del tmp[k] # not useful to tf log these values = to_tf_values(tmp, ["ray", "tune"]) train_stats = tf.Summary(value=values) t = result.get(TIMESTEPS_TOTAL) or result[TRAINING_ITERATION] @@ -158,15 +163,21 @@ class _VisKitLogger(Logger): def _init(self): """CSV outputted with Headers as first set of results.""" # Note that we assume params.json was already created by JsonLogger - self._file = open(os.path.join(self.logdir, "progress.csv"), "w") + progress_file = os.path.join(self.logdir, "progress.csv") + self._continuing = os.path.exists(progress_file) + self._file = open(progress_file, "a") self._csv_out = None def on_result(self, result): if self._csv_out is None: self._csv_out = csv.DictWriter(self._file, result.keys()) - self._csv_out.writeheader() + if not self._continuing: + self._csv_out.writeheader() self._csv_out.writerow(result.copy()) + def flush(self): + self._file.flush() + def close(self): self._file.close() From e2b83807764fab1e009d26ad0cf5ec6f5887c6ec Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 12 Nov 2018 15:50:34 -0800 Subject: [PATCH 42/74] Redo checkpointing from trial runner --- python/ray/test/cluster_utils.py | 1 + python/ray/tune/test/cluster_tests.py | 44 +++++++++++++++------------ python/ray/tune/trial.py | 19 ++++++++++++ python/ray/tune/trial_executor.py | 9 ++++++ python/ray/tune/trial_runner.py | 27 ++++++++-------- 5 files changed, 69 insertions(+), 31 deletions(-) diff --git a/python/ray/test/cluster_utils.py b/python/ray/test/cluster_utils.py index c468cf01584b..6a3fa165b98f 100644 --- a/python/ray/test/cluster_utils.py +++ b/python/ray/test/cluster_utils.py @@ -39,6 +39,7 @@ def __init__(self, if connect: redis_password = head_node_args.get("redis_password") output_info = ray.init( + ignore_reinit_error=True, redis_address=self.redis_address, redis_password=redis_password) logger.info(output_info) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 48f567d8542b..c1de03335997 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -35,11 +35,8 @@ def _restore(self, state): self.state = state -@pytest.fixture -def start_connected_cluster(): - # Start the Ray processes. - - cluster = Cluster( +def _start_new_cluster(): + return Cluster( initialize_head=True, connect=True, head_node_args={ @@ -48,6 +45,12 @@ def start_connected_cluster(): "num_heartbeats_timeout": 10 }) }) + + +@pytest.fixture +def start_connected_cluster(): + # Start the Ray processes. + cluster = _start_new_cluster() yield cluster # The code after the yield will run as teardown code. ray.shutdown() @@ -74,8 +77,10 @@ def start_connected_emptyhead_cluster(): cluster.shutdown() -@pytest.mark.skipif(pytest_timeout is None, reason="Timeout package" - " not installed; skipping test that may hang.") +@pytest.mark.skipif( + pytest_timeout is None, + reason="Timeout package" + " not installed; skipping test that may hang.") @pytest.mark.timeout(10, method="thread") def test_counting_resources(start_connected_cluster): """Tests that Tune accounting is consistent with actual cluster.""" @@ -116,8 +121,10 @@ def test_counting_resources(start_connected_cluster): @pytest.mark.skip("Add this test once reconstruction is fixed") -@pytest.mark.skipif(pytest_timeout is None, reason="Timeout package" - " not installed; skipping test that may hang.") +@pytest.mark.skipif( + pytest_timeout is None, + reason="Timeout package" + " not installed; skipping test that may hang.") @pytest.mark.timeout(10, method="thread") def test_remove_node_before_result(start_connected_cluster): """Removing a node should cause a Trial to be requeued.""" @@ -125,11 +132,7 @@ def test_remove_node_before_result(start_connected_cluster): node = cluster.add_node(resources=dict(CPU=1)) runner = TrialRunner(BasicVariantGenerator()) - kwargs = { - "stopping_criterion": { - "training_iteration": 3 - } - } + kwargs = {"stopping_criterion": {"training_iteration": 3}} tune.register_trainable("test", _Train) trials = [Trial("test", **kwargs), Trial("test", **kwargs)] @@ -258,9 +261,9 @@ def test_trial_requeue(start_connected_emptyhead_cluster): runner.step() -def test_cluster_down(start_connected_emptyhead_cluster): +def test_cluster_down_simple(): """Removing a node in full cluster causes Trial to be requeued.""" - cluster = start_connected_emptyhead_cluster + cluster = _start_new_cluster() node = cluster.add_node(resources=dict(CPU=1)) runner = TrialRunner(BasicVariantGenerator()) @@ -279,8 +282,11 @@ def test_cluster_down(start_connected_emptyhead_cluster): runner.step() # start runner.step() # 1 result - checkpoint = runner.checkpoint() + checkpoint_dir = runner.save() cluster.shutdown() - _start_new_cluster() - runner = TrialRunner.from_checkpoint(checkpoint) + ray.shutdown() + + cluster = _start_new_cluster() + runner = TrialRunner(BasicVariantGenerator()) + runner.restore(checkpoint_dir) runner.step() diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index 8e0526c7aab8..ab904f455a71 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -326,3 +326,22 @@ def __str__(self): if self.experiment_tag: identifier += "_" + self.experiment_tag return identifier + + def __getstate__(self): + if not self._checkpoint.storage == Checkpoint.DISK: + raise ValueError("Most recent checkpoint cannot be in-memory.") + state = self.__dict__.copy() + # Remove the unpicklable entries. + if state["result_logger"]: + state["result_logger"].flush() + state["_logger_started"] = True + + state["result_logger"] = None + state["runner"] = None + return state + + def __setstate__(self, state): + logger_started = state.pop("_logger_started") + self.__dict__.update(state) + if logger_started: + self.init_logger() diff --git a/python/ray/tune/trial_executor.py b/python/ray/tune/trial_executor.py index bd36a84761e5..de8da35e54fa 100644 --- a/python/ray/tune/trial_executor.py +++ b/python/ray/tune/trial_executor.py @@ -188,3 +188,12 @@ def save(self, trial, storage=Checkpoint.DISK): """ raise NotImplementedError("Subclasses of TrialExecutor must provide " "save() method") + + def recreate_trial_from_checkpoint(self, checkpoint): + """Restores a trial to its checkpointed state.""" + assert checkpoint.storage == Checkpoint.DISK + with open(os.path.join(checkpoint.value, "trial.ckpt"), "rb") as f: + trial = pickle.load(f) + if trial.status == Trial.RUNNING: + trial.status = Trial.PENDING + return trial diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index e80b11ac912e..35b3888f07e2 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -12,7 +12,7 @@ from ray.tune import TuneError from ray.tune.ray_trial_executor import RayTrialExecutor -from ray.tune.result import TIME_THIS_ITER_S +from ray.tune.result import TIME_THIS_ITER_S, DEFAULT_RESULTS_DIR from ray.tune.trial import Trial from ray.tune.schedulers import FIFOScheduler, TrialScheduler from ray.tune.web_server import TuneServer @@ -91,33 +91,33 @@ def __init__(self, self._stop_queue = [] self._verbose = verbose self._queue_trials = queue_trials - self._checkpoints = {} + self._trial_checkpoints = {} - def save(self, checkpoint_dir): + def save(self, checkpoint_dir=DEFAULT_RESULTS_DIR): # search_alg_checkpoint = self._search_alg.save(checkpoint_dir) # scheduler_alg_checkpoint = self._scheduler_alg.save(checkpoint_dir) runner_state = { - "checkpoints": pickle.dumps(list(self._checkpoints.values())), + "checkpoints": list(self._trial_checkpoints.values()), "total_time": self._total_time, "stop_queue": self._stop_queue } with open(os.path.join(checkpoint_dir, "experiment.state"), "wb") as f: pickle.dump(runner_state, f) + return checkpoint_dir + def restore(self, checkpoint_dir): logger.debug("Stopping all trials.") for trial in self._trials: self.stop_trial(trial) with open(os.path.join(checkpoint_dir, "experiment.state"), "rb") as f: - state = pickle.load(f) + runner_state = pickle.load(f) logger.info("Replacing all trials with checkpoint state.") - runner_state = state[0] - checkpoints = pickle.loads(runner_state["checkpoints"]) - - for ckpt in checkpoints: - self.add_trial(self.trial_executor.recreate_from_checkpoint(ckpt)) + self._trial_checkpoints = runner_state["checkpoints"] + for ckpt in self._trial_checkpoints: + self.add_trial(pickle.loads(ckpt)) self._total_time = runner_state["total_time"] self._stop_queue = runner_state["stop_queue"] @@ -317,8 +317,11 @@ def _process_events(self): def _checkpoint_if_needed(self, trial, result): if trial.should_checkpoint(result): # TODO(rliaw): This is a blocking call - checkpoint = self.trial_executor.save_trial(trial) - self._checkpoints[trial] = checkpoint + self.trial_executor.save(trial, storage=Checkpoint.DISK) + try: + self._checkpoints[trial] = pickle.dumps(trial) + except ValueError: + logger.exception("Error checkpointing full trial state.") def _try_recover(self, trial, error_msg): try: From 7aab84feed8eb2f54028f59128a78b7cf8565910 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 14 Nov 2018 12:56:05 -0800 Subject: [PATCH 43/74] fix up tests and checkpointing --- python/ray/test/cluster_utils.py | 7 +------ python/ray/tune/trial.py | 11 +++-------- 2 files changed, 4 insertions(+), 14 deletions(-) diff --git a/python/ray/test/cluster_utils.py b/python/ray/test/cluster_utils.py index 2f3c04e251af..85f3f8c972c6 100644 --- a/python/ray/test/cluster_utils.py +++ b/python/ray/test/cluster_utils.py @@ -172,16 +172,11 @@ def shutdown(self): all_nodes = list(self.worker_nodes) for node in all_nodes: self.remove_node(node) -<<<<<<< HEAD + if self.head_node: self.remove_node(self.head_node) else: logger.warning("No headnode exists!") -======= - - if self.head_node is not None: - self.remove_node(self.head_node) ->>>>>>> 57c7b4238ef02e61de256feaa77f8028a28d630d class Node(object): diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index 8e0526c7aab8..65683eeb53c7 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -281,15 +281,10 @@ def has_checkpoint(self): def should_recover(self): """Returns whether the trial qualifies for restoring. - This is if a checkpoint frequency is set, or if there exists - a pre-existing checkpoint. + This is if a checkpoint frequency is set, which includes settings + where there may not yet be a checkpoint. """ - if self.checkpoint_freq > 0: - # Edge case of beginning trial - if (self.checkpoint_freq > self.last_result[TRAINING_ITERATION] - or self.last_result is None): - return True - return self.has_checkpoint() + return self.checkpoint_freq > 0 def update_last_result(self, result, terminate=False): if terminate: From 1e8a33d25b4794c057e6d9572b921e4fba5ba405 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 14 Nov 2018 14:39:06 -0800 Subject: [PATCH 44/74] import error --- python/ray/test/test_global_state.py | 2 +- python/ray/tune/test/cluster_tests.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/python/ray/test/test_global_state.py b/python/ray/test/test_global_state.py index fc6aa0bc5929..68805a8ec561 100644 --- a/python/ray/test/test_global_state.py +++ b/python/ray/test/test_global_state.py @@ -6,7 +6,7 @@ import pytest try: import pytest_timeout -except ModuleNotFoundError: +except ImportError: pytest_timeout = None import time diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 4697ac69b443..57aeed34fedb 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -7,7 +7,7 @@ import pytest try: import pytest_timeout -except ModuleNotFoundError as e: +except ImportError: pytest_timeout = None from ray.test.cluster_utils import Cluster From 637e7075d60cd419370cb004c00a8de2c090b555 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Thu, 15 Nov 2018 14:20:51 -0800 Subject: [PATCH 45/74] timeout? --- docker/examples/Dockerfile | 2 +- python/ray/tune/test/cluster_tests.py | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/docker/examples/Dockerfile b/docker/examples/Dockerfile index 80685b7d3154..9cdee4ff117e 100644 --- a/docker/examples/Dockerfile +++ b/docker/examples/Dockerfile @@ -5,7 +5,7 @@ FROM ray-project/deploy # This updates numpy to 1.14 and mutes errors from other libraries RUN conda install -y numpy RUN apt-get install -y zlib1g-dev -RUN pip install gym[atari] opencv-python==3.2.0.8 tensorflow lz4 keras +RUN pip install gym[atari] opencv-python==3.2.0.8 tensorflow lz4 keras pytest-timeout RUN pip install -U h5py # Mutes FutureWarnings RUN pip install --upgrade git+git://github.com/hyperopt/hyperopt.git RUN conda install pytorch-cpu torchvision-cpu -c pytorch diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 57aeed34fedb..19bb101105bb 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -156,6 +156,9 @@ def test_remove_node_before_result(start_connected_cluster): runner.step() +@pytest.mark.skipif(pytest_timeout is None, reason="Timeout package" + " not installed; skipping test that may hang.") +@pytest.mark.timeout(120, method="thread") def test_trial_migration(start_connected_emptyhead_cluster): """Removing a node while cluster has space should migrate trial. @@ -226,6 +229,9 @@ def test_trial_migration(start_connected_emptyhead_cluster): runner.step() +@pytest.mark.skipif(pytest_timeout is None, reason="Timeout package" + " not installed; skipping test that may hang.") +@pytest.mark.timeout(120, method="thread") def test_trial_requeue(start_connected_emptyhead_cluster): """Removing a node in full cluster causes Trial to be requeued.""" cluster = start_connected_emptyhead_cluster From 162b308d684cf9ccefbdea70e92c593319bb6537 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Thu, 15 Nov 2018 14:23:24 -0800 Subject: [PATCH 46/74] lint --- python/ray/tune/test/cluster_tests.py | 26 +++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 19bb101105bb..e0f37844aa91 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -74,8 +74,9 @@ def start_connected_emptyhead_cluster(): cluster.shutdown() -@pytest.mark.skipif(pytest_timeout is None, reason="Timeout package" - " not installed; skipping test that may hang.") +@pytest.mark.skipif( + pytest_timeout is None, + reason="Timeout package not installed; skipping test.") @pytest.mark.timeout(10, method="thread") def test_counting_resources(start_connected_cluster): """Tests that Tune accounting is consistent with actual cluster.""" @@ -116,8 +117,9 @@ def test_counting_resources(start_connected_cluster): @pytest.mark.skip("Add this test once reconstruction is fixed") -@pytest.mark.skipif(pytest_timeout is None, reason="Timeout package" - " not installed; skipping test that may hang.") +@pytest.mark.skipif( + pytest_timeout is None, + reason="Timeout package not installed; skipping test.") @pytest.mark.timeout(10, method="thread") def test_remove_node_before_result(start_connected_cluster): """Removing a node should cause a Trial to be requeued.""" @@ -125,11 +127,7 @@ def test_remove_node_before_result(start_connected_cluster): node = cluster.add_node(resources=dict(CPU=1)) runner = TrialRunner(BasicVariantGenerator()) - kwargs = { - "stopping_criterion": { - "training_iteration": 3 - } - } + kwargs = {"stopping_criterion": {"training_iteration": 3}} tune.register_trainable("test", _Train) trials = [Trial("test", **kwargs), Trial("test", **kwargs)] @@ -156,8 +154,9 @@ def test_remove_node_before_result(start_connected_cluster): runner.step() -@pytest.mark.skipif(pytest_timeout is None, reason="Timeout package" - " not installed; skipping test that may hang.") +@pytest.mark.skipif( + pytest_timeout is None, + reason="Timeout package not installed; skipping test.") @pytest.mark.timeout(120, method="thread") def test_trial_migration(start_connected_emptyhead_cluster): """Removing a node while cluster has space should migrate trial. @@ -229,8 +228,9 @@ def test_trial_migration(start_connected_emptyhead_cluster): runner.step() -@pytest.mark.skipif(pytest_timeout is None, reason="Timeout package" - " not installed; skipping test that may hang.") +@pytest.mark.skipif( + pytest_timeout is None, + reason="Timeout package not installed; skipping test.") @pytest.mark.timeout(120, method="thread") def test_trial_requeue(start_connected_emptyhead_cluster): """Removing a node in full cluster causes Trial to be requeued.""" From e683224e7e212aa3392d03c46c8d2a96a073a07f Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Sun, 18 Nov 2018 22:09:16 -0800 Subject: [PATCH 47/74] Checkpoint and tests --- python/ray/tune/test/cluster_tests.py | 28 ++++++++++++------- python/ray/tune/trial.py | 12 ++++++--- python/ray/tune/trial_runner.py | 39 +++++++++++++++++++-------- python/ray/tune/tune.py | 10 ++++++- 4 files changed, 64 insertions(+), 25 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 8de720882644..83140ad7c4b7 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -4,6 +4,7 @@ import json import time +import tempfile import pytest try: import pytest_timeout @@ -267,15 +268,15 @@ def test_trial_requeue(start_connected_emptyhead_cluster): runner.step() -def test_cluster_down_simple(): +def test_cluster_down_simple(start_connected_cluster): """Removing a node in full cluster causes Trial to be requeued.""" - cluster = _start_new_cluster() - node = cluster.add_node(resources=dict(CPU=1)) - - runner = TrialRunner(BasicVariantGenerator()) + cluster = start_connected_cluster + cluster.add_node(resources=dict(CPU=1)) + tmpdir = tempfile.mkdtemp() + runner = TrialRunner(BasicVariantGenerator(), checkpoint_dir=tmpdir) kwargs = { "stopping_criterion": { - "training_iteration": 5 + "training_iteration": 2 }, "checkpoint_freq": 1, "max_failures": 1 @@ -287,12 +288,21 @@ def test_cluster_down_simple(): runner.add_trial(t) runner.step() # start - runner.step() # 1 result - checkpoint_dir = runner.save() + runner.step() # start2 + assert all(t.status == Trial.RUNNING for t in runner.get_trials()) + runner.save() + cluster.shutdown() ray.shutdown() cluster = _start_new_cluster() + tune.register_trainable("test", _Train) runner = TrialRunner(BasicVariantGenerator()) - runner.restore(checkpoint_dir) + runner.restore(tmpdir) + runner.step() runner.step() + + for i in range(4): + runner.step() + + assert all(t.status == Trial.TERMINATED for t in runner.get_trials()) diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index 3639322e1109..f51cd29a2ef0 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -216,17 +216,19 @@ def should_stop(self, result): return False - def should_checkpoint(self, result): + def should_checkpoint(self): """Whether this trial is due for checkpointing.""" + result = self.last_result or {} if result.get(DONE) and self.checkpoint_at_end: return True - if not self.checkpoint_freq: + if self.checkpoint_freq: + return result.get( + TRAINING_ITERATION, 0) % self.checkpoint_freq == 0 + else: return False - return self.last_result[TRAINING_ITERATION] % self.checkpoint_freq == 0 - def progress_string(self): """Returns a progress message for printing out to the console.""" @@ -330,6 +332,8 @@ def __getstate__(self): if state["result_logger"]: state["result_logger"].flush() state["_logger_started"] = True + else: + state["_logger_started"] = False state["result_logger"] = None state["runner"] = None diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index 3bdfcccc3aaf..b777d675cd9c 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -13,7 +13,7 @@ from ray.tune import TuneError from ray.tune.ray_trial_executor import RayTrialExecutor from ray.tune.result import TIME_THIS_ITER_S, DEFAULT_RESULTS_DIR -from ray.tune.trial import Trial +from ray.tune.trial import Trial, Checkpoint from ray.tune.schedulers import FIFOScheduler, TrialScheduler from ray.tune.web_server import TuneServer @@ -54,6 +54,8 @@ def __init__(self, search_alg, scheduler=None, launch_web_server=False, + checkpoint_dir=None, + checkpoint_freq=None, server_port=TuneServer.DEFAULT_PORT, verbose=True, queue_trials=False, @@ -85,15 +87,20 @@ def __init__(self, self._global_time_limit = float( os.environ.get("TRIALRUNNER_WALLTIME_LIMIT", float('inf'))) self._total_time = 0 + self._iteration = 0 self._server = None if launch_web_server: self._server = TuneServer(self, server_port) self._stop_queue = [] self._verbose = verbose self._queue_trials = queue_trials + self._checkpoint_dir = checkpoint_dir + self._checkpoint_freq = checkpoint_freq self._trial_checkpoints = {} - def save(self, checkpoint_dir=DEFAULT_RESULTS_DIR): + def save(self, checkpoint_dir=None): + if checkpoint_dir is None: + checkpoint_dir = self._checkpoint_dir # search_alg_checkpoint = self._search_alg.save(checkpoint_dir) # scheduler_alg_checkpoint = self._scheduler_alg.save(checkpoint_dir) runner_state = { @@ -115,8 +122,8 @@ def restore(self, checkpoint_dir): runner_state = pickle.load(f) logger.info("Replacing all trials with checkpoint state.") - self._trial_checkpoints = runner_state["checkpoints"] - for ckpt in self._trial_checkpoints: + for ckpt in runner_state["checkpoints"]: + # NOTE: This will repickle the current trial state self.add_trial(pickle.loads(ckpt)) self._total_time = runner_state["total_time"] @@ -147,6 +154,11 @@ def step(self): self.trial_executor.start_trial(next_trial) elif self.trial_executor.get_running_trials(): self._process_events() + if self._checkpoint_freq: + if self._iteration % self._checkpoint_freq == 0: + self.save(self._checkpoint_dir) + + self._iteration += 1 else: for trial in self._trials: if trial.status == Trial.PENDING: @@ -196,6 +208,7 @@ def add_trial(self, trial): """ trial.set_verbose(self._verbose) self._scheduler_alg.on_trial_add(self, trial) + self._checkpoint_if_needed(trial) self._trials.append(trial) def debug_string(self, max_debug=MAX_DEBUG_TRIALS): @@ -309,14 +322,14 @@ def _process_events(self): result, terminate=(decision == TrialScheduler.STOP)) if decision == TrialScheduler.CONTINUE: - self._checkpoint_if_needed(trial, result) + self._checkpoint_if_needed(trial) self.trial_executor.continue_training(trial) elif decision == TrialScheduler.PAUSE: self.trial_executor.pause_trial(trial) elif decision == TrialScheduler.STOP: # Checkpoint before ending the trial # if checkpoint_at_end experiment option is set to True - self._checkpoint_if_needed(trial, result) + self._checkpoint_if_needed(trial) self.trial_executor.stop_trial(trial) else: assert False, "Invalid scheduling decision: {}".format( @@ -334,12 +347,16 @@ def _process_events(self): trial.trial_id, error=True) self.trial_executor.stop_trial(trial, True, error_msg) - def _checkpoint_if_needed(self, trial, result): - if trial.should_checkpoint(result): - # TODO(rliaw): This is a blocking call - self.trial_executor.save(trial, storage=Checkpoint.DISK) + def _checkpoint_if_needed(self, trial): + """Checkpoints trial based off trial.last_result.""" + if trial.should_checkpoint(): + + # Save trial runtime if possible + if hasattr(trial, "runner") and trial.runner: + self.trial_executor.save(trial, storage=Checkpoint.DISK) + try: - self._checkpoints[trial] = pickle.dumps(trial) + self._trial_checkpoints[trial] = pickle.dumps(trial) except ValueError: logger.exception("Error checkpointing full trial state.") diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index 335660ecb836..b8a98d9424bf 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -35,6 +35,9 @@ def _make_scheduler(args): def run_experiments(experiments=None, search_alg=None, scheduler=None, + restore_from_path=None, + checkpoint_dir=None, + checkpoint_freq=None, with_server=False, server_port=TuneServer.DEFAULT_PORT, verbose=True, @@ -95,14 +98,19 @@ def run_experiments(experiments=None, runner = TrialRunner( search_alg, scheduler=scheduler, + checkpoint_dir=checkpoint_dir, + checkpoint_freq=checkpoint_freq, launch_web_server=with_server, server_port=server_port, verbose=verbose, queue_trials=queue_trials, trial_executor=trial_executor) - logger.info(runner.debug_string(max_debug=99999)) + # TODO(rliaw): Have better explicit designation for restoring. + if restore_from_path and os.path.exists(restore_from_path): + runner.restore(restore_from_path) + logger.info(runner.debug_string(max_debug=99999)) last_debug = 0 while not runner.is_finished(): runner.step() From ef67acf4bb0c7690c1f34d55d4bd8cbc41dd78f1 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Sun, 18 Nov 2018 23:48:04 -0800 Subject: [PATCH 48/74] one full cluster failure --- python/ray/tune/test/cluster_tests.py | 84 ++++++++++++++++++++++++++- python/ray/tune/trial.py | 5 ++ python/ray/tune/trial_executor.py | 9 --- python/ray/tune/trial_runner.py | 4 +- python/ray/tune/tune.py | 1 + 5 files changed, 90 insertions(+), 13 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 83140ad7c4b7..1b3e256feb0d 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -3,6 +3,7 @@ from __future__ import print_function import json +import os import time import tempfile import pytest @@ -12,6 +13,7 @@ pytest_timeout = None from ray.test.cluster_utils import Cluster +from ray.test.test_utils import run_string_as_driver_nonblocking, run_string_as_driver import ray from ray import tune from ray.tune.error import TuneError @@ -289,6 +291,7 @@ def test_cluster_down_simple(start_connected_cluster): runner.step() # start runner.step() # start2 + runner.step() # step assert all(t.status == Trial.RUNNING for t in runner.get_trials()) runner.save() @@ -299,10 +302,85 @@ def test_cluster_down_simple(start_connected_cluster): tune.register_trainable("test", _Train) runner = TrialRunner(BasicVariantGenerator()) runner.restore(tmpdir) - runner.step() - runner.step() + print([t.status for t in runner.get_trials()]) + runner.step() # start + runner.step() # start2 + + for i in range(3): + runner.step() - for i in range(4): + with pytest.raises(TuneError): runner.step() assert all(t.status == Trial.TERMINATED for t in runner.get_trials()) + + +def test_cluster_down_full(start_connected_cluster): + cluster = start_connected_cluster + + tmpdir = tempfile.mkdtemp() + + trainable_str = """ +import time + +class _Train(tune.Trainable): + def _setup(self, config): + self.state = dict(hi=1) + + def _train(self): + self.state["hi"] += 1 + time.sleep(0.5) + return dict() + + def _save(self, path): + return self.state + + def _restore(self, state): + self.state = state + +tune.register_trainable("train", _Train) +""" + script = """ +import os +import ray +from ray import tune + + +ray.init(redis_address="{redis_address}") + +{register_trainable_script} + +kwargs = dict( + run="train", + stop=dict(training_iteration=2), + checkpoint_freq=1, + max_failures=1) + +tune.run_experiments( + dict(experiment=kwargs), + checkpoint_dir="{checkpoint_dir}", + checkpoint_freq=3) +""".format( + redis_address=cluster.redis_address, + checkpoint_dir=tmpdir, + register_trainable_script=trainable_str) + run_string_as_driver_nonblocking(script) + while not os.path.exists(os.path.join(tmpdir, "experiment.state")): + time.sleep(0.5) + cluster.shutdown() + + cluster = _start_new_cluster() + script = """ +import ray +from ray import tune + +ray.init(redis_address="{redis_address}") + +{register_trainable_script} + +tune.run_experiments(restore_from_path="{restore_path}") +""".format( + redis_address=cluster.redis_address, + restore_path=tmpdir, + register_trainable_script=trainable_str) + run_string_as_driver(script) diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index f51cd29a2ef0..bd825ac1d525 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -328,6 +328,11 @@ def __getstate__(self): if not self._checkpoint.storage == Checkpoint.DISK: raise ValueError("Most recent checkpoint cannot be in-memory.") state = self.__dict__.copy() + + if state["status"] == Trial.RUNNING: + # TODO(rliaw): Consider using _try_recover to keep this state + # when pickling. + state["status"] = Trial.PENDING # Remove the unpicklable entries. if state["result_logger"]: state["result_logger"].flush() diff --git a/python/ray/tune/trial_executor.py b/python/ray/tune/trial_executor.py index b2266dd294ad..e0b541218bf1 100644 --- a/python/ray/tune/trial_executor.py +++ b/python/ray/tune/trial_executor.py @@ -192,12 +192,3 @@ def save(self, trial, storage=Checkpoint.DISK): """ raise NotImplementedError("Subclasses of TrialExecutor must provide " "save() method") - - def recreate_trial_from_checkpoint(self, checkpoint): - """Restores a trial to its checkpointed state.""" - assert checkpoint.storage == Checkpoint.DISK - with open(os.path.join(checkpoint.value, "trial.ckpt"), "rb") as f: - trial = pickle.load(f) - if trial.status == Trial.RUNNING: - trial.status = Trial.PENDING - return trial diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index b777d675cd9c..ca1a85153025 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -12,7 +12,7 @@ from ray.tune import TuneError from ray.tune.ray_trial_executor import RayTrialExecutor -from ray.tune.result import TIME_THIS_ITER_S, DEFAULT_RESULTS_DIR +from ray.tune.result import TIME_THIS_ITER_S from ray.tune.trial import Trial, Checkpoint from ray.tune.schedulers import FIFOScheduler, TrialScheduler from ray.tune.web_server import TuneServer @@ -101,6 +101,8 @@ def __init__(self, def save(self, checkpoint_dir=None): if checkpoint_dir is None: checkpoint_dir = self._checkpoint_dir + if not os.path.exists(checkpoint_dir): + os.makedirs(checkpoint_dir) # search_alg_checkpoint = self._search_alg.save(checkpoint_dir) # scheduler_alg_checkpoint = self._scheduler_alg.save(checkpoint_dir) runner_state = { diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index b8a98d9424bf..a13ce945a48b 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -3,6 +3,7 @@ from __future__ import print_function import logging +import os import time from ray.tune.error import TuneError From 0541f9220e597c6dbb0694052ef8e930dd20348f Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 20 Nov 2018 23:27:46 -0800 Subject: [PATCH 49/74] lint --- python/ray/tune/test/cluster_tests.py | 36 +++++++++++++-------------- 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index e0f37844aa91..f9425cc3e301 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -19,20 +19,23 @@ from ray.tune.suggest import BasicVariantGenerator -class _Train(tune.Trainable): - def _setup(self, config): - self.state = {"hi": 1} +def register_test_trainable(): + class _Train(tune.Trainable): + def _setup(self, config): + self.state = {"hi": 1} - def _train(self): - self.state["hi"] += 1 - time.sleep(0.5) - return {} + def _train(self): + self.state["hi"] += 1 + time.sleep(0.5) + return {} - def _save(self, path): - return self.state + def _save(self, path): + return self.state - def _restore(self, state): - self.state = state + def _restore(self, state): + self.state = state + + tune.register_trainable("test", _Train) @pytest.fixture @@ -48,6 +51,7 @@ def start_connected_cluster(): "num_heartbeats_timeout": 10 }) }) + register_test_trainable() yield cluster # The code after the yield will run as teardown code. ray.shutdown() @@ -67,7 +71,7 @@ def start_connected_emptyhead_cluster(): "num_heartbeats_timeout": 10 }) }) - tune.register_trainable("test", _Train) + register_test_trainable() yield cluster # The code after the yield will run as teardown code. ray.shutdown() @@ -91,7 +95,6 @@ def test_counting_resources(start_connected_cluster): runner = TrialRunner(BasicVariantGenerator()) kwargs = {"stopping_criterion": {"training_iteration": 10}} - tune.register_trainable("test", _Train) trials = [Trial("test", **kwargs), Trial("test", **kwargs)] for t in trials: runner.add_trial(t) @@ -125,11 +128,11 @@ def test_remove_node_before_result(start_connected_cluster): """Removing a node should cause a Trial to be requeued.""" cluster = start_connected_cluster node = cluster.add_node(resources=dict(CPU=1)) + # TODO(rliaw): Make blocking an option? + assert cluster.wait_for_nodes() runner = TrialRunner(BasicVariantGenerator()) kwargs = {"stopping_criterion": {"training_iteration": 3}} - - tune.register_trainable("test", _Train) trials = [Trial("test", **kwargs), Trial("test", **kwargs)] for t in trials: runner.add_trial(t) @@ -139,7 +142,6 @@ def test_remove_node_before_result(start_connected_cluster): assert all(t.status == Trial.RUNNING for t in trials) runner.step() # 1 result - print(runner.debug_string()) cluster.remove_node(node) cluster.wait_for_nodes() @@ -246,14 +248,12 @@ def test_trial_requeue(start_connected_emptyhead_cluster): "max_failures": 1 } - tune.register_trainable("test", _Train) trials = [Trial("test", **kwargs), Trial("test", **kwargs)] for t in trials: runner.add_trial(t) runner.step() # start runner.step() # 1 result - print(runner.debug_string()) cluster.remove_node(node) assert cluster.wait_for_nodes() From b884fedf575fe536ba0b0004d9ff2f1d61a25226 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 21 Nov 2018 00:52:15 -0800 Subject: [PATCH 50/74] Add better test --- python/ray/tune/test/cluster_tests.py | 52 +++++++-------------------- 1 file changed, 12 insertions(+), 40 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 3866d5c5b96b..325c762eded8 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -2,6 +2,7 @@ from __future__ import division from __future__ import print_function +import inspect import json import os import time @@ -12,10 +13,10 @@ except ImportError: pytest_timeout = None -from ray.test.cluster_utils import Cluster -from ray.test.test_utils import run_string_as_driver_nonblocking, run_string_as_driver import ray from ray import tune +from ray.test.cluster_utils import Cluster +from ray.test.test_utils import run_string_as_driver_nonblocking from ray.tune.error import TuneError from ray.tune.trial import Trial from ray.tune.trial_runner import TrialRunner @@ -319,26 +320,6 @@ def test_cluster_down_full(start_connected_cluster): tmpdir = tempfile.mkdtemp() - trainable_str = """ -import time - -class _Train(tune.Trainable): - def _setup(self, config): - self.state = dict(hi=1) - - def _train(self): - self.state["hi"] += 1 - time.sleep(0.5) - return dict() - - def _save(self, path): - return self.state - - def _restore(self, state): - self.state = state - -tune.register_trainable("train", _Train) -""" script = """ import os import ray @@ -347,10 +328,11 @@ def _restore(self, state): ray.init(redis_address="{redis_address}") -{register_trainable_script} +{register_trainable_fn} +{run_register_trainable_fn}() kwargs = dict( - run="train", + run="test", stop=dict(training_iteration=2), checkpoint_freq=1, max_failures=1) @@ -362,24 +344,14 @@ def _restore(self, state): """.format( redis_address=cluster.redis_address, checkpoint_dir=tmpdir, - register_trainable_script=trainable_str) + register_trainable_fn=inspect.getsource(register_test_trainable), + run_register_trainable_fn=register_test_trainable.__name__) run_string_as_driver_nonblocking(script) while not os.path.exists(os.path.join(tmpdir, "experiment.state")): time.sleep(0.5) + ray.shutdown() cluster.shutdown() - cluster = _start_new_cluster() - script = """ -import ray -from ray import tune - -ray.init(redis_address="{redis_address}") - -{register_trainable_script} - -tune.run_experiments(restore_from_path="{restore_path}") -""".format( - redis_address=cluster.redis_address, - restore_path=tmpdir, - register_trainable_script=trainable_str) - run_string_as_driver(script) + register_test_trainable() + trials = tune.run_experiments(restore_from_path=tmpdir) + assert all(t.status == Trial.TERMINATED for t in trials) From 8617f2de12689fe3bf6897600b5f571641156e74 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 21 Nov 2018 16:49:07 -0800 Subject: [PATCH 51/74] error test --- python/ray/tune/test/cluster_tests.py | 91 +++++++++++++++++++++++++-- 1 file changed, 86 insertions(+), 5 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 325c762eded8..39dfaab3ab93 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -42,6 +42,27 @@ def _restore(self, state): tune.register_trainable("test", _Train) +def register_fail_trainable(): + class _Train(tune.Trainable): + def _setup(self, config): + self.state = {"hi": 1} + + def _train(self): + self.state["hi"] += 1 + time.sleep(0.5) + if self.state["hi"] % 2 == 1: + assert False + return {} + + def _save(self, path): + return self.state + + def _restore(self, state): + self.state = state + + tune.register_trainable("fail", _Train) + + def _start_new_cluster(): return Cluster( initialize_head=True, @@ -315,13 +336,12 @@ def test_cluster_down_simple(start_connected_cluster): assert all(t.status == Trial.TERMINATED for t in runner.get_trials()) -def test_cluster_down_full(start_connected_cluster): +def test_cluster_down_full(start_connected_cluster, tmpdir): cluster = start_connected_cluster - tmpdir = tempfile.mkdtemp() - script = """ import os +import time import ray from ray import tune @@ -333,7 +353,7 @@ def test_cluster_down_full(start_connected_cluster): kwargs = dict( run="test", - stop=dict(training_iteration=2), + stop=dict(training_iteration=3), checkpoint_freq=1, max_failures=1) @@ -347,11 +367,72 @@ def test_cluster_down_full(start_connected_cluster): register_trainable_fn=inspect.getsource(register_test_trainable), run_register_trainable_fn=register_test_trainable.__name__) run_string_as_driver_nonblocking(script) + while not os.path.exists(os.path.join(tmpdir, "experiment.state")): - time.sleep(0.5) + time.sleep(0.1) ray.shutdown() cluster.shutdown() cluster = _start_new_cluster() register_test_trainable() + + # Check that last_result.iteration = 1 + runner = TrialRunner(BasicVariantGenerator()) + runner.restore(tmpdir) + trials = runner.get_trials() + assert trials[0].last_result["training_iteration"] == 1 + trials = tune.run_experiments(restore_from_path=tmpdir) assert all(t.status == Trial.TERMINATED for t in trials) + + +def test_cluster_down_error(start_connected_cluster, tmpdir): + cluster = start_connected_cluster + + script = """ +import os +import time +import ray +from ray import tune + + +ray.init(redis_address="{redis_address}") + +{register_trainable_fn} +{run_register_trainable_fn}() + +kwargs = dict( + run="fail", + stop=dict(training_iteration=5), + checkpoint_freq=1, + max_failures=1) + +tune.run_experiments( + dict(experiment1=kwargs), + checkpoint_dir="{checkpoint_dir}", + checkpoint_freq=3) +""".format( + redis_address=cluster.redis_address, + checkpoint_dir=tmpdir, + register_trainable_fn=inspect.getsource(register_fail_trainable), + run_register_trainable_fn=register_fail_trainable.__name__) + run_string_as_driver_nonblocking(script) + + while not os.path.exists(os.path.join(tmpdir, "experiment.state")): + time.sleep(0.1) + ray.shutdown() + cluster.shutdown() + + cluster = _start_new_cluster() + register_fail_trainable() + + # Inspect the internal trialrunner + runner = TrialRunner(BasicVariantGenerator()) + runner.restore(tmpdir) + trials = runner.get_trials() + assert trials[0].last_result["training_iteration"] == 1 + assert trials[0].status == Trial.PENDING + + # Restore properly from checkpoint + trials = tune.run_experiments( + restore_from_path=tmpdir, raise_on_failed_trial=False) + assert all(t.status == Trial.ERROR for t in trials) From f7e31bd10e18a1a2e5c81eaf00cfff8002e18392 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Sat, 24 Nov 2018 01:35:14 -0800 Subject: [PATCH 52/74] some docs --- python/ray/tune/trial.py | 2 -- python/ray/tune/trial_runner.py | 20 ++++++++++++-------- python/ray/tune/tune.py | 7 ++++++- 3 files changed, 18 insertions(+), 11 deletions(-) diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index bd825ac1d525..337f4e2b02e7 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -330,8 +330,6 @@ def __getstate__(self): state = self.__dict__.copy() if state["status"] == Trial.RUNNING: - # TODO(rliaw): Consider using _try_recover to keep this state - # when pickling. state["status"] = Trial.PENDING # Remove the unpicklable entries. if state["result_logger"]: diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index ca1a85153025..cde9cfbe2d80 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -55,7 +55,7 @@ def __init__(self, scheduler=None, launch_web_server=False, checkpoint_dir=None, - checkpoint_freq=None, + checkpoint_freq=0, server_port=TuneServer.DEFAULT_PORT, verbose=True, queue_trials=False, @@ -67,6 +67,9 @@ def __init__(self, Trial objects. scheduler (TrialScheduler): Defaults to FIFOScheduler. launch_web_server (bool): Flag for starting TuneServer + checkpoint_dir (str): Path where global checkpoints are stored. + checkpoint_freq (int): How many trial results between global + checkpoints. A value of 0 (default) disables checkpointing. server_port (int): Port number for launching TuneServer verbose (bool): Flag for verbosity. If False, trial results will not be output. @@ -98,11 +101,12 @@ def __init__(self, self._checkpoint_freq = checkpoint_freq self._trial_checkpoints = {} - def save(self, checkpoint_dir=None): - if checkpoint_dir is None: - checkpoint_dir = self._checkpoint_dir - if not os.path.exists(checkpoint_dir): - os.makedirs(checkpoint_dir) + def save(self): + """Saves all trial checkpoints""" + checkpoint_dir = self._checkpoint_dir + if not os.path.exists(checkpoint_dir): + logger.debug("Checkpoint directory newly created.") + os.makedirs(checkpoint_dir) # search_alg_checkpoint = self._search_alg.save(checkpoint_dir) # scheduler_alg_checkpoint = self._scheduler_alg.save(checkpoint_dir) runner_state = { @@ -125,8 +129,8 @@ def restore(self, checkpoint_dir): logger.info("Replacing all trials with checkpoint state.") for ckpt in runner_state["checkpoints"]: - # NOTE: This will repickle the current trial state - self.add_trial(pickle.loads(ckpt)) + trial = pickle.loads(ckpt) + self.add_trial(trial) self._total_time = runner_state["total_time"] self._stop_queue = runner_state["stop_queue"] diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index a13ce945a48b..d77fd2b79c84 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -38,7 +38,7 @@ def run_experiments(experiments=None, scheduler=None, restore_from_path=None, checkpoint_dir=None, - checkpoint_freq=None, + checkpoint_freq=0, with_server=False, server_port=TuneServer.DEFAULT_PORT, verbose=True, @@ -55,6 +55,11 @@ def run_experiments(experiments=None, scheduler (TrialScheduler): Scheduler for executing the experiment. Choose among FIFO (default), MedianStopping, AsyncHyperBand, and HyperBand. + restore_from_path (str): Restores experiment execution state to + given checkpoint path. + checkpoint_dir (str): Path at which experiment checkpoints are stored. + checkpoint_freq (int): How many trial results between + checkpoints. A value of 0 (default) disables checkpointing. with_server (bool): Starts a background Tune server. Needed for using the Client API. server_port (int): Port number for launching TuneServer. From a2355f87c0c180a20286c6844c62fc2f3b4c57cc Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Sat, 24 Nov 2018 20:00:27 -0800 Subject: [PATCH 53/74] Tests and better recovery handling --- python/ray/tune/ray_trial_executor.py | 28 +++++-- .../ray/tune/test/ray_trial_executor_test.py | 12 ++- python/ray/tune/test/trial_runner_test.py | 73 ++++++++++++++++--- python/ray/tune/trial.py | 8 +- python/ray/tune/trial_executor.py | 29 +------- python/ray/tune/trial_runner.py | 53 +++++++++++--- python/ray/tune/tune.py | 12 ++- 7 files changed, 152 insertions(+), 63 deletions(-) diff --git a/python/ray/tune/ray_trial_executor.py b/python/ray/tune/ray_trial_executor.py index 7b725e05f342..dbee770b44cd 100644 --- a/python/ray/tune/ray_trial_executor.py +++ b/python/ray/tune/ray_trial_executor.py @@ -110,25 +110,39 @@ def _stop_trial(self, trial, error=False, error_msg=None, if stop_logger: trial.close_logger() - def start_trial(self, trial, checkpoint_obj=None): - """Starts the trial.""" + def start_trial(self, trial, checkpoint=None, raise_on_failure=False): + """Starts the trial. + + Will not return resources if trial repeatedly fails on start. + + Args: + trial (Trial): Trial to be started. + checkpoint(Checkpoint): A Python object or path storing the state + of trial. + raise_on_failure (bool): To raise exception on failure in starting. + + Raises: + Exception after 1 retries if `raise_on_failure` is True. + """ self._commit_resources(trial.resources) try: - self._start_trial(trial, checkpoint_obj) + self._start_trial(trial, checkpoint) except Exception: logger.exception("Error stopping runner - retrying...") error_msg = traceback.format_exc() time.sleep(2) self._stop_trial(trial, error=True, error_msg=error_msg) try: - self._start_trial(trial) - except Exception: + self._start_trial(trial, checkpoint) + except Exception as exc: logger.exception("Error starting runner, aborting!") error_msg = traceback.format_exc() self._stop_trial(trial, error=True, error_msg=error_msg) # note that we don't return the resources, since they may # have been lost + if raise_on_failure: + raise exc def _find_item(self, dictionary, item): out = [rid for rid, t in dictionary.items() if t is item] @@ -140,11 +154,13 @@ def stop_trial(self, trial, error=False, error_msg=None, stop_logger=True): self._stop_trial( trial, error=error, error_msg=error_msg, stop_logger=stop_logger) if prior_status == Trial.RUNNING: - self._return_resources(trial.resources) out = self._find_item(self._running, trial) for result_id in out: self._running.pop(result_id) + logger.debug("Returning resources for this trial.") + self._return_resources(trial.resources) + def continue_training(self, trial): """Continues the training of this trial.""" diff --git a/python/ray/tune/test/ray_trial_executor_test.py b/python/ray/tune/test/ray_trial_executor_test.py index 35c413e717bb..8e6ef765cee3 100644 --- a/python/ray/tune/test/ray_trial_executor_test.py +++ b/python/ray/tune/test/ray_trial_executor_test.py @@ -9,8 +9,9 @@ from ray.rllib import _register_all from ray.tune import Trainable from ray.tune.ray_trial_executor import RayTrialExecutor +from ray.tune.registry import _global_registry, TRAINABLE_CLASS from ray.tune.suggest import BasicVariantGenerator -from ray.tune.trial import Trial, Checkpoint +from ray.tune.trial import Trial, Checkpoint, Resources class RayTrialExecutorTest(unittest.TestCase): @@ -50,6 +51,15 @@ def testPauseResume(self): self.trial_executor.stop_trial(trial) self.assertEqual(Trial.TERMINATED, trial.status) + def testStartFailure(self): + _global_registry.register(TRAINABLE_CLASS, "asdf", None) + trial = Trial("asdf", resources=Resources(1, 0)) + self.trial_executor.start_trial(trial) + self.assertEqual(Trial.ERROR, trial.status) + self.assertRaises( + Exception, lambda: self.trial_executor.start_trial( + trial, raise_on_error=True)) + def testPauseResume2(self): """Tests that pausing works for trials being processed.""" trial = Trial("__fake") diff --git a/python/ray/tune/test/trial_runner_test.py b/python/ray/tune/test/trial_runner_test.py index 6b142d354ec7..6af30e87fd3a 100644 --- a/python/ray/tune/test/trial_runner_test.py +++ b/python/ray/tune/test/trial_runner_test.py @@ -5,6 +5,7 @@ import os import time import unittest +from unittest import mock import ray from ray.rllib import _register_all @@ -845,6 +846,23 @@ def testMaxConcurrentSuggestions(self): self.assertEqual(len(searcher.next_trials()), 0) +def create_mock_components(): + + class _MockScheduler(FIFOScheduler): + errored_trials = [] + def on_trial_error(self, trial_runner, trial): + self.errored_trials += [trial] + + class _MockSearchAlg(BasicVariantGenerator): + errored_trials = [] + def on_trial_complete(self, trial_id, error=False, **kwargs): + if error: + self.errored_trials += [trial_id] + + searchalg = _MockSearchAlg() + scheduler = _MockScheduler() + return searchalg, scheduler + class TrialRunnerTest(unittest.TestCase): def tearDown(self): ray.shutdown() @@ -889,16 +907,6 @@ def train(config, reporter): self.assertLessEqual(len(trial.logdir), 200) trial_executor.stop_trial(trial) - def testTrialErrorOnStart(self): - ray.init() - trial_executor = RayTrialExecutor() - _global_registry.register(TRAINABLE_CLASS, "asdf", None) - trial = Trial("asdf", resources=Resources(1, 0)) - try: - trial_executor.start_trial(trial) - except Exception as e: - self.assertIn("a class", str(e)) - def testExtraResources(self): ray.init(num_cpus=4, num_gpus=2) runner = TrialRunner(BasicVariantGenerator()) @@ -1055,7 +1063,9 @@ def testThrowOnOverstep(self): def testFailureRecoveryDisabled(self): ray.init(num_cpus=1, num_gpus=1) - runner = TrialRunner(BasicVariantGenerator()) + searchalg, scheduler = create_mock_components() + + runner = TrialRunner(searchalg, scheduler=scheduler) kwargs = { "resources": Resources(cpu=1, gpu=1), "checkpoint_freq": 1, @@ -1074,10 +1084,15 @@ def testFailureRecoveryDisabled(self): runner.step() self.assertEqual(trials[0].status, Trial.ERROR) self.assertEqual(trials[0].num_failures, 1) + self.assertEqual(len(searchalg.errored_trials), 1) + self.assertEqual(len(scheduler.errored_trials), 1) def testFailureRecoveryEnabled(self): ray.init(num_cpus=1, num_gpus=1) - runner = TrialRunner(BasicVariantGenerator()) + searchalg, scheduler = create_mock_components() + + runner = TrialRunner(searchalg, scheduler=scheduler) + kwargs = { "resources": Resources(cpu=1, gpu=1), "checkpoint_freq": 1, @@ -1098,6 +1113,40 @@ def testFailureRecoveryEnabled(self): self.assertEqual(trials[0].num_failures, 1) runner.step() self.assertEqual(trials[0].status, Trial.RUNNING) + self.assertEqual(len(searchalg.errored_trials), 0) + self.assertEqual(len(scheduler.errored_trials), 0) + + def testFailureRecoveryNodeRemoval(self): + ray.init(num_cpus=1, num_gpus=1) + searchalg, scheduler = create_mock_components() + + runner = TrialRunner(searchalg, scheduler=scheduler) + + kwargs = { + "resources": Resources(cpu=1, gpu=1), + "checkpoint_freq": 1, + "max_failures": 1, + "config": { + "mock_error": True, + }, + } + runner.add_trial(Trial("__fake", **kwargs)) + trials = runner.get_trials() + + with mock.patch('ray.global_state.cluster_resources') as res_mock: + res_mock.return_value = {"CPU": 1, "GPU": 1} + runner.step() + self.assertEqual(trials[0].status, Trial.RUNNING) + runner.step() + self.assertEqual(trials[0].status, Trial.RUNNING) + + # Mimic a node failure + res_mock.return_value = {"CPU": 0, "GPU": 0} + runner.step() + self.assertEqual(trials[0].status, Trial.PENDING) + self.assertEqual(trials[0].num_failures, 1) + self.assertEqual(len(searchalg.errored_trials), 0) + self.assertEqual(len(scheduler.errored_trials), 1) def testFailureRecoveryMaxFailures(self): ray.init(num_cpus=1, num_gpus=1) diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index 337f4e2b02e7..1a9e4ff17a30 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -283,10 +283,12 @@ def has_checkpoint(self): def should_recover(self): """Returns whether the trial qualifies for restoring. - This is if a checkpoint frequency is set, which includes settings - where there may not yet be a checkpoint. + This is if a checkpoint frequency is set and has not failed more than + max_failures. This may return true even when there may not yet + be a checkpoint. """ - return self.checkpoint_freq > 0 + return (self.checkpoint_freq > 0 + and self.num_failures < self.max_failures) def update_last_result(self, result, terminate=False): if terminate: diff --git a/python/ray/tune/trial_executor.py b/python/ray/tune/trial_executor.py index e0b541218bf1..d67e7705a1b1 100644 --- a/python/ray/tune/trial_executor.py +++ b/python/ray/tune/trial_executor.py @@ -31,15 +31,14 @@ def has_resources(self, resources): raise NotImplementedError("Subclasses of TrialExecutor must provide " "has_resources() method") - def start_trial(self, trial, checkpoint=None): - """Starts the trial restoring from checkpoint if checkpoint != None. - - If an error is encountered when starting the trial, an exception will - be thrown. + def start_trial(self, trial, checkpoint=None, raise_on_failure=False): + """Starts the trial restoring from checkpoint if checkpoint is provided. Args: + trial (Trial): Trial to be started. checkpoint(Checkpoint): A Python object or path storing the state of trial. + raise_on_failure (bool): To raise exception on failure in starting. """ raise NotImplementedError("Subclasses of TrialExecutor must provide " "start_trial() method") @@ -59,26 +58,6 @@ def stop_trial(self, trial, error=False, error_msg=None, stop_logger=True): raise NotImplementedError("Subclasses of TrialExecutor must provide " "stop_trial() method") - def restart_trial(self, trial, error_msg=None): - """Restarts or requeues the trial. - - The state of the trial should restore from the last checkpoint. Trial - is requeued if the cluster no longer has resources to accomodate it. - - Args: - error_msg (str): Optional error message. - """ - self.stop_trial( - trial, - error=error_msg is not None, - error_msg=error_msg, - stop_logger=False) - trial.result_logger.flush() - if self.has_resources(trial.resources): - self.start_trial(trial) - else: - trial.status = Trial.PENDING - def continue_training(self, trial): """Continues the training of this trial.""" pass diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index cde9cfbe2d80..f1f0e9455d32 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -12,7 +12,7 @@ from ray.tune import TuneError from ray.tune.ray_trial_executor import RayTrialExecutor -from ray.tune.result import TIME_THIS_ITER_S +from ray.tune.result import TIME_THIS_ITER_S, DEFAULT_RESULTS_DIR from ray.tune.trial import Trial, Checkpoint from ray.tune.schedulers import FIFOScheduler, TrialScheduler from ray.tune.web_server import TuneServer @@ -162,7 +162,7 @@ def step(self): self._process_events() if self._checkpoint_freq: if self._iteration % self._checkpoint_freq == 0: - self.save(self._checkpoint_dir) + self.save() self._iteration += 1 else: @@ -344,14 +344,17 @@ def _process_events(self): logger.exception("Error processing event.") error_msg = traceback.format_exc() if trial.status == Trial.RUNNING: - if trial.should_recover() and \ - trial.num_failures < trial.max_failures: - self._try_recover(trial, error_msg) + if trial.should_recover(): + self.try_recover(trial, error_msg) else: + self.trial_executor.stop_trial( + trial, + error=error_msg is not None, + error_msg=error_msg) self._scheduler_alg.on_trial_error(self, trial) self._search_alg.on_trial_complete( trial.trial_id, error=True) - self.trial_executor.stop_trial(trial, True, error_msg) + def _checkpoint_if_needed(self, trial): """Checkpoints trial based off trial.last_result.""" @@ -366,15 +369,41 @@ def _checkpoint_if_needed(self, trial): except ValueError: logger.exception("Error checkpointing full trial state.") - def _try_recover(self, trial, error_msg): + def try_recover(self, trial, error_msg): + """Tries to recover trial. + + Notifies SearchAlgorithm and Scheduler if failure to recover. + """ try: - logger.info("Attempting to recover" - " trial state from last checkpoint.") - self.trial_executor.restart_trial(trial, error_msg) + self.trial_executor.stop_trial( + trial, + error=error_msg is not None, + error_msg=error_msg, + stop_logger=False) + trial.result_logger.flush() + if self.trial_executor.has_resources(trial.resources): + logger.info("Attempting to recover" + " trial state from last checkpoint.") + self.trial_executor.start_trial(trial, raise_on_failure=True) + else: + logger.debug("Notifying Scheduler and requeueing trial.") + self._requeue_trial(trial) except Exception: error_msg = traceback.format_exc() - logger.warning("Error recovering trial from checkpoint, abort.") - self.trial_executor.stop_trial(trial, True, error_msg=error_msg) + logger.exception("Error recovering trial from checkpoint, abort.") + self._scheduler_alg.on_trial_error(self, trial) + self._search_alg.on_trial_complete( + trial.trial_id, error=True) + + def _requeue_trial(self, trial): + """Notification to TrialScheduler and requeue trial. + + This does not notify the SearchAlgorithm because + the function evaluation is still in progress. + """ + self._scheduler_alg.on_trial_error(self, trial) + trial.status = Trial.PENDING + self._scheduler_alg.on_trial_add(self, trial) def _update_trial_queue(self, blocking=False, timeout=600): """Adds next trials to queue if possible. diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index d77fd2b79c84..afefc03fc19f 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -9,6 +9,7 @@ from ray.tune.error import TuneError from ray.tune.suggest import BasicVariantGenerator from ray.tune.trial import Trial, DEBUG_PRINT_INTERVAL +from ray.tune.result import DEFAULT_RESULTS_DIR from ray.tune.log_sync import wait_for_log_sync from ray.tune.trial_runner import TrialRunner from ray.tune.schedulers import (HyperBandScheduler, AsyncHyperBandScheduler, @@ -58,6 +59,7 @@ def run_experiments(experiments=None, restore_from_path (str): Restores experiment execution state to given checkpoint path. checkpoint_dir (str): Path at which experiment checkpoints are stored. + Defaults to DEFAULT_RESULTS_DIR. checkpoint_freq (int): How many trial results between checkpoints. A value of 0 (default) disables checkpointing. with_server (bool): Starts a background Tune server. Needed for @@ -99,12 +101,10 @@ def run_experiments(experiments=None, if search_alg is None: search_alg = BasicVariantGenerator() - search_alg.add_configurations(experiments) - runner = TrialRunner( search_alg, scheduler=scheduler, - checkpoint_dir=checkpoint_dir, + checkpoint_dir=checkpoint_dir or DEFAULT_RESULTS_DIR, checkpoint_freq=checkpoint_freq, launch_web_server=with_server, server_port=server_port, @@ -112,9 +112,13 @@ def run_experiments(experiments=None, queue_trials=queue_trials, trial_executor=trial_executor) - # TODO(rliaw): Have better explicit designation for restoring. if restore_from_path and os.path.exists(restore_from_path): + assert experiments is None, ( + "Simultaneous starting experiments and restoring not supported.") runner.restore(restore_from_path) + else: + search_alg.add_configurations(experiments) + logger.info(runner.debug_string(max_debug=99999)) last_debug = 0 From 5513099f50e6b2d12eb0e7be62b0518828cb19a0 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Sat, 24 Nov 2018 22:34:32 -0800 Subject: [PATCH 54/74] Add unit test for restoring (but currently failing --- python/ray/tune/test/cluster_tests.py | 1 + python/ray/tune/test/trial_runner_test.py | 65 +++++++++++++++++++++++ python/ray/tune/trial_runner.py | 6 +-- 3 files changed, 69 insertions(+), 3 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 39dfaab3ab93..5af6a5503aec 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -266,6 +266,7 @@ def test_trial_requeue(start_connected_emptyhead_cluster): """Removing a node in full cluster causes Trial to be requeued.""" cluster = start_connected_emptyhead_cluster node = cluster.add_node(resources=dict(CPU=1)) + assert cluster.wait_for_nodes() runner = TrialRunner(BasicVariantGenerator()) kwargs = { diff --git a/python/ray/tune/test/trial_runner_test.py b/python/ray/tune/test/trial_runner_test.py index 6af30e87fd3a..55582c8d8395 100644 --- a/python/ray/tune/test/trial_runner_test.py +++ b/python/ray/tune/test/trial_runner_test.py @@ -3,6 +3,8 @@ from __future__ import print_function import os +import shutil +import tempfile import time import unittest from unittest import mock @@ -1548,6 +1550,69 @@ def _suggest(self, trial_id): self.assertTrue(searcher.is_finished()) self.assertRaises(TuneError, runner.step) + def testSaveRestore(self): + ray.init(num_cpus=3) + tmpdir = tempfile.mkdtemp() + default_resources = Resources(cpu=1, gpu=0) + + runner = TrialRunner( + BasicVariantGenerator(), + checkpoint_dir=tmpdir, + checkpoint_freq=1) + trials = [Trial("__fake", + trial_id="trial_terminate", + stopping_criterion={"training_iteration": 1}, + checkpoint_freq=1, + checkpoint_at_end=True, + resources=default_resources)] + runner.add_trial(trials[0]) + runner.step() # start + runner.step() + self.assertEquals(trials[0].status, Trial.TERMINATED) + + trials += [ + Trial("__fake", + trial_id="trial_fail", + stopping_criterion={"training_iteration": 3}, + checkpoint_freq=1, + checkpoint_at_end=True, + config={"mock_error": True}, + resources=default_resources)] + runner.add_trial(trials[1]) + runner.step() + runner.step() + runner.step() + self.assertEquals(trials[1].status, Trial.ERROR) + + trials += [ + Trial("__fake", + trial_id="trial_succ", + stopping_criterion={"training_iteration": 2}, + checkpoint_freq=1, + resources=default_resources)] + runner.add_trial(trials[2]) + runner.step() + self.assertEquals(trials[2].status, Trial.RUNNING) + + runner2 = TrialRunner(BasicVariantGenerator()) + runner2.restore(tmpdir) + for tid in ["trial_terminate", "trial_fail"]: + original_trial = runner.get_trial(tid) + restored_trial = runner2.get_trial(tid) + self.assertEqual(original_trial.status, restored_trial.status) + + restored_trial = runner2.get_trial("trial_succ") + self.assertEqual(Trial.PENDING, restored_trial.status) + + runner2.step() + runner2.step() + runner2.step() + self.assertRaises(TuneError, runner2.step) + shutil.rmtree(tmpdir) + + + + if __name__ == "__main__": unittest.main(verbosity=2) diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index f1f0e9455d32..f5ccc34db637 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -107,6 +107,7 @@ def save(self): if not os.path.exists(checkpoint_dir): logger.debug("Checkpoint directory newly created.") os.makedirs(checkpoint_dir) + logger.warning("Search Algorithm and Scheduler not checkpointed.") # search_alg_checkpoint = self._search_alg.save(checkpoint_dir) # scheduler_alg_checkpoint = self._scheduler_alg.save(checkpoint_dir) runner_state = { @@ -359,7 +360,6 @@ def _process_events(self): def _checkpoint_if_needed(self, trial): """Checkpoints trial based off trial.last_result.""" if trial.should_checkpoint(): - # Save trial runtime if possible if hasattr(trial, "runner") and trial.runner: self.trial_executor.save(trial, storage=Checkpoint.DISK) @@ -398,8 +398,8 @@ def try_recover(self, trial, error_msg): def _requeue_trial(self, trial): """Notification to TrialScheduler and requeue trial. - This does not notify the SearchAlgorithm because - the function evaluation is still in progress. + This does not notify the SearchAlgorithm because the function + evaluation is still in progress. """ self._scheduler_alg.on_trial_error(self, trial) trial.status = Trial.PENDING From 782f194b8b72d9a10290fca2c5714d4675710ec8 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Sun, 25 Nov 2018 00:41:49 -0800 Subject: [PATCH 55/74] pickle if needed when you set status --- python/ray/tune/ray_trial_executor.py | 17 +++++++------ python/ray/tune/test/trial_runner_test.py | 8 +++--- python/ray/tune/trial_executor.py | 30 ++++++++++++++++++++--- python/ray/tune/trial_runner.py | 28 ++++++++++----------- 4 files changed, 52 insertions(+), 31 deletions(-) diff --git a/python/ray/tune/ray_trial_executor.py b/python/ray/tune/ray_trial_executor.py index dbee770b44cd..6f8595d5c92b 100644 --- a/python/ray/tune/ray_trial_executor.py +++ b/python/ray/tune/ray_trial_executor.py @@ -5,6 +5,7 @@ import logging import os +import pickle import time import traceback @@ -19,8 +20,8 @@ class RayTrialExecutor(TrialExecutor): """An implemention of TrialExecutor based on Ray.""" - def __init__(self, queue_trials=False): - super(RayTrialExecutor, self).__init__(queue_trials) + def __init__(self, queue_trials=False, track_checkpoints=False): + super(RayTrialExecutor, self).__init__(queue_trials, track_checkpoints) self._running = {} # Since trial resume after paused should not run # trial.train.remote(), thus no more new remote object id generated. @@ -60,7 +61,7 @@ def _train(self, trial): def _start_trial(self, trial, checkpoint=None): prior_status = trial.status - trial.status = Trial.RUNNING + self.set_status(trial, Trial.RUNNING) trial.runner = self._setup_runner(trial) if not self.restore(trial, checkpoint): return @@ -88,9 +89,9 @@ def _stop_trial(self, trial, error=False, error_msg=None, """ if error: - trial.status = Trial.ERROR + self.set_status(trial, Trial.ERROR) else: - trial.status = Trial.TERMINATED + self.set_status(trial, Trial.TERMINATED) try: trial.write_error_log(error_msg) @@ -103,7 +104,7 @@ def _stop_trial(self, trial, error=False, error_msg=None, stop_tasks, num_returns=2, timeout=250) except Exception: logger.exception("Error stopping runner.") - trial.status = Trial.ERROR + self.set_status(trial, Trial.ERROR) finally: trial.runner = None @@ -309,7 +310,7 @@ def restore(self, trial, checkpoint=None): return True if trial.runner is None: logger.error("Unable to restore - no runner.") - trial.status = Trial.ERROR + self.set_status(trial, Trial.ERROR) return False try: value = checkpoint.value @@ -323,5 +324,5 @@ def restore(self, trial, checkpoint=None): return True except Exception: logger.exception("Error restoring runner.") - trial.status = Trial.ERROR + self.set_status(trial, Trial.ERROR) return False diff --git a/python/ray/tune/test/trial_runner_test.py b/python/ray/tune/test/trial_runner_test.py index 55582c8d8395..1992e437e294 100644 --- a/python/ray/tune/test/trial_runner_test.py +++ b/python/ray/tune/test/trial_runner_test.py @@ -849,14 +849,15 @@ def testMaxConcurrentSuggestions(self): def create_mock_components(): - class _MockScheduler(FIFOScheduler): errored_trials = [] + def on_trial_error(self, trial_runner, trial): self.errored_trials += [trial] class _MockSearchAlg(BasicVariantGenerator): errored_trials = [] + def on_trial_complete(self, trial_id, error=False, **kwargs): if error: self.errored_trials += [trial_id] @@ -865,6 +866,7 @@ def on_trial_complete(self, trial_id, error=False, **kwargs): scheduler = _MockScheduler() return searchalg, scheduler + class TrialRunnerTest(unittest.TestCase): def tearDown(self): ray.shutdown() @@ -1592,6 +1594,7 @@ def testSaveRestore(self): resources=default_resources)] runner.add_trial(trials[2]) runner.step() + self.assertEquals(len(runner.trial_executor.get_checkpoints()), 3) self.assertEquals(trials[2].status, Trial.RUNNING) runner2 = TrialRunner(BasicVariantGenerator()) @@ -1611,8 +1614,5 @@ def testSaveRestore(self): shutil.rmtree(tmpdir) - - - if __name__ == "__main__": unittest.main(verbosity=2) diff --git a/python/ray/tune/trial_executor.py b/python/ray/tune/trial_executor.py index d67e7705a1b1..92410906153f 100644 --- a/python/ray/tune/trial_executor.py +++ b/python/ray/tune/trial_executor.py @@ -4,6 +4,7 @@ from __future__ import print_function import logging +import pickle from ray.tune.trial import Trial, Checkpoint @@ -15,7 +16,7 @@ class TrialExecutor(object): and starting/stopping trials. """ - def __init__(self, queue_trials=False): + def __init__(self, queue_trials=False, track_checkpoints=False): """Initializes a new TrialExecutor. Args: @@ -25,6 +26,27 @@ def __init__(self, queue_trials=False): automatic scale-up. """ self._queue_trials = queue_trials + self._track_checkpoints = track_checkpoints + self._checkpoints = {} + + def set_status(self, trial, status): + trial.status = status + self.checkpoint_metadata_if_needed(trial) + + def checkpoint_metadata_if_needed(self, trial): + if self._track_checkpoints: + if trial._checkpoint.storage == Checkpoint.MEMORY: + logger.debug("Not saving data for trial w/ memory checkpoint.") + return + try: + logger.debug("Saving trial metadata.") + metadata = pickle.dumps(trial) + self._checkpoints[trial.trial_id] = metadata + except ValueError: + logger.exception("Error checkpointing trial metadata.") + + def get_checkpoints(self): + return self._checkpoints.copy() def has_resources(self, resources): """Returns whether this runner has at least the specified resources.""" @@ -72,15 +94,15 @@ def pause_trial(self, trial): try: self.save(trial, Checkpoint.MEMORY) self.stop_trial(trial, stop_logger=False) - trial.status = Trial.PAUSED + self.set_status(trial, Trial.PAUSED) except Exception: logger.exception("Error pausing runner.") - trial.status = Trial.ERROR + self.set_status(trial, Trial.ERROR) def unpause_trial(self, trial): """Sets PAUSED trial to pending to allow scheduler to start.""" assert trial.status == Trial.PAUSED, trial.status - trial.status = Trial.PENDING + self.set_status(trial, Trial.PENDING) def resume_trial(self, trial): """Resumes PAUSED trials. This is a blocking call.""" diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index f5ccc34db637..ce0be05aa039 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -5,10 +5,10 @@ import collections import logging import os +import pickle import re import time import traceback -import pickle from ray.tune import TuneError from ray.tune.ray_trial_executor import RayTrialExecutor @@ -83,7 +83,8 @@ def __init__(self, self._scheduler_alg = scheduler or FIFOScheduler() self._trials = [] self.trial_executor = trial_executor or \ - RayTrialExecutor(queue_trials=queue_trials) + RayTrialExecutor(queue_trials=queue_trials, + track_checkpoints=checkpoint_freq > 0) # For debugging, it may be useful to halt trials after some time has # elapsed. TODO(ekl) consider exposing this in the API. @@ -111,7 +112,7 @@ def save(self): # search_alg_checkpoint = self._search_alg.save(checkpoint_dir) # scheduler_alg_checkpoint = self._scheduler_alg.save(checkpoint_dir) runner_state = { - "checkpoints": list(self._trial_checkpoints.values()), + "checkpoints": list(self.trial_executor.get_checkpoints().values()), "total_time": self._total_time, "stop_queue": self._stop_queue } @@ -161,11 +162,6 @@ def step(self): self.trial_executor.start_trial(next_trial) elif self.trial_executor.get_running_trials(): self._process_events() - if self._checkpoint_freq: - if self._iteration % self._checkpoint_freq == 0: - self.save() - - self._iteration += 1 else: for trial in self._trials: if trial.status == Trial.PENDING: @@ -186,6 +182,13 @@ def step(self): "There are paused trials, but no more pending " "trials with sufficient resources.") + if self._checkpoint_freq: + if self._iteration % self._checkpoint_freq == 0: + self.save() + + self._iteration += 1 + + if self._server: self._process_requests() @@ -356,18 +359,13 @@ def _process_events(self): self._search_alg.on_trial_complete( trial.trial_id, error=True) - def _checkpoint_if_needed(self, trial): """Checkpoints trial based off trial.last_result.""" if trial.should_checkpoint(): # Save trial runtime if possible if hasattr(trial, "runner") and trial.runner: self.trial_executor.save(trial, storage=Checkpoint.DISK) - - try: - self._trial_checkpoints[trial] = pickle.dumps(trial) - except ValueError: - logger.exception("Error checkpointing full trial state.") + self.trial_executor.checkpoint_metadata_if_needed(trial) def try_recover(self, trial, error_msg): """Tries to recover trial. @@ -402,7 +400,7 @@ def _requeue_trial(self, trial): evaluation is still in progress. """ self._scheduler_alg.on_trial_error(self, trial) - trial.status = Trial.PENDING + self.trial_executor.set_status(trial, Trial.PENDING) self._scheduler_alg.on_trial_add(self, trial) def _update_trial_queue(self, blocking=False, timeout=600): From d1d5a56088b5e281c6514efe70803bd751476423 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Sun, 25 Nov 2018 00:50:01 -0800 Subject: [PATCH 56/74] yapf --- python/ray/tune/ray_trial_executor.py | 5 +-- python/ray/tune/test/trial_runner_test.py | 47 +++++++++++++---------- python/ray/tune/trial.py | 4 +- python/ray/tune/trial_runner.py | 7 ++-- python/ray/tune/tune.py | 1 - 5 files changed, 33 insertions(+), 31 deletions(-) diff --git a/python/ray/tune/ray_trial_executor.py b/python/ray/tune/ray_trial_executor.py index 6f8595d5c92b..11cbcd1d3fce 100644 --- a/python/ray/tune/ray_trial_executor.py +++ b/python/ray/tune/ray_trial_executor.py @@ -155,13 +155,12 @@ def stop_trial(self, trial, error=False, error_msg=None, stop_logger=True): self._stop_trial( trial, error=error, error_msg=error_msg, stop_logger=stop_logger) if prior_status == Trial.RUNNING: + logger.debug("Returning resources for this trial.") + self._return_resources(trial.resources) out = self._find_item(self._running, trial) for result_id in out: self._running.pop(result_id) - logger.debug("Returning resources for this trial.") - self._return_resources(trial.resources) - def continue_training(self, trial): """Continues the training of this trial.""" diff --git a/python/ray/tune/test/trial_runner_test.py b/python/ray/tune/test/trial_runner_test.py index 1992e437e294..2b9d6ff42fc4 100644 --- a/python/ray/tune/test/trial_runner_test.py +++ b/python/ray/tune/test/trial_runner_test.py @@ -1558,28 +1558,31 @@ def testSaveRestore(self): default_resources = Resources(cpu=1, gpu=0) runner = TrialRunner( - BasicVariantGenerator(), - checkpoint_dir=tmpdir, - checkpoint_freq=1) - trials = [Trial("__fake", - trial_id="trial_terminate", - stopping_criterion={"training_iteration": 1}, - checkpoint_freq=1, - checkpoint_at_end=True, - resources=default_resources)] + BasicVariantGenerator(), checkpoint_dir=tmpdir, checkpoint_freq=1) + trials = [ + Trial( + "__fake", + trial_id="trial_terminate", + stopping_criterion={"training_iteration": 1}, + checkpoint_freq=1, + checkpoint_at_end=True, + resources=default_resources) + ] runner.add_trial(trials[0]) runner.step() # start runner.step() self.assertEquals(trials[0].status, Trial.TERMINATED) trials += [ - Trial("__fake", - trial_id="trial_fail", - stopping_criterion={"training_iteration": 3}, - checkpoint_freq=1, - checkpoint_at_end=True, - config={"mock_error": True}, - resources=default_resources)] + Trial( + "__fake", + trial_id="trial_fail", + stopping_criterion={"training_iteration": 3}, + checkpoint_freq=1, + checkpoint_at_end=True, + config={"mock_error": True}, + resources=default_resources) + ] runner.add_trial(trials[1]) runner.step() runner.step() @@ -1587,11 +1590,13 @@ def testSaveRestore(self): self.assertEquals(trials[1].status, Trial.ERROR) trials += [ - Trial("__fake", - trial_id="trial_succ", - stopping_criterion={"training_iteration": 2}, - checkpoint_freq=1, - resources=default_resources)] + Trial( + "__fake", + trial_id="trial_succ", + stopping_criterion={"training_iteration": 2}, + checkpoint_freq=1, + resources=default_resources) + ] runner.add_trial(trials[2]) runner.step() self.assertEquals(len(runner.trial_executor.get_checkpoints()), 3) diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index 1a9e4ff17a30..e555e9512101 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -224,8 +224,8 @@ def should_checkpoint(self): return True if self.checkpoint_freq: - return result.get( - TRAINING_ITERATION, 0) % self.checkpoint_freq == 0 + return result.get(TRAINING_ITERATION, + 0) % self.checkpoint_freq == 0 else: return False diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index ce0be05aa039..bec7bee90ae0 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -112,7 +112,8 @@ def save(self): # search_alg_checkpoint = self._search_alg.save(checkpoint_dir) # scheduler_alg_checkpoint = self._scheduler_alg.save(checkpoint_dir) runner_state = { - "checkpoints": list(self.trial_executor.get_checkpoints().values()), + "checkpoints": list( + self.trial_executor.get_checkpoints().values()), "total_time": self._total_time, "stop_queue": self._stop_queue } @@ -188,7 +189,6 @@ def step(self): self._iteration += 1 - if self._server: self._process_requests() @@ -390,8 +390,7 @@ def try_recover(self, trial, error_msg): error_msg = traceback.format_exc() logger.exception("Error recovering trial from checkpoint, abort.") self._scheduler_alg.on_trial_error(self, trial) - self._search_alg.on_trial_complete( - trial.trial_id, error=True) + self._search_alg.on_trial_complete(trial.trial_id, error=True) def _requeue_trial(self, trial): """Notification to TrialScheduler and requeue trial. diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index afefc03fc19f..328a4543ac07 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -119,7 +119,6 @@ def run_experiments(experiments=None, else: search_alg.add_configurations(experiments) - logger.info(runner.debug_string(max_debug=99999)) last_debug = 0 while not runner.is_finished(): From be445e8295bafbb22184adc4926b6c72d18b152c Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Sun, 25 Nov 2018 01:26:01 -0800 Subject: [PATCH 57/74] docs and small test for nosaving --- python/ray/tune/ray_trial_executor.py | 2 +- python/ray/tune/test/trial_runner_test.py | 70 ++++++++++++++--------- python/ray/tune/trial_executor.py | 21 ++++++- 3 files changed, 64 insertions(+), 29 deletions(-) diff --git a/python/ray/tune/ray_trial_executor.py b/python/ray/tune/ray_trial_executor.py index 11cbcd1d3fce..538376a5b5b3 100644 --- a/python/ray/tune/ray_trial_executor.py +++ b/python/ray/tune/ray_trial_executor.py @@ -118,7 +118,7 @@ def start_trial(self, trial, checkpoint=None, raise_on_failure=False): Args: trial (Trial): Trial to be started. - checkpoint(Checkpoint): A Python object or path storing the state + checkpoint (Checkpoint): A Python object or path storing the state of trial. raise_on_failure (bool): To raise exception on failure in starting. diff --git a/python/ray/tune/test/trial_runner_test.py b/python/ray/tune/test/trial_runner_test.py index 2b9d6ff42fc4..e47c2e83f208 100644 --- a/python/ray/tune/test/trial_runner_test.py +++ b/python/ray/tune/test/trial_runner_test.py @@ -1553,36 +1553,32 @@ def _suggest(self, trial_id): self.assertRaises(TuneError, runner.step) def testSaveRestore(self): + """Creates trials of different status to test runner.save/restore.""" ray.init(num_cpus=3) tmpdir = tempfile.mkdtemp() default_resources = Resources(cpu=1, gpu=0) runner = TrialRunner( - BasicVariantGenerator(), checkpoint_dir=tmpdir, checkpoint_freq=1) - trials = [ - Trial( - "__fake", - trial_id="trial_terminate", - stopping_criterion={"training_iteration": 1}, - checkpoint_freq=1, - checkpoint_at_end=True, - resources=default_resources) - ] + BasicVariantGenerator(), + checkpoint_dir=tmpdir, + checkpoint_freq=1) + trials = [Trial("__fake", + trial_id="trial_terminate", + stopping_criterion={"training_iteration": 1}, + checkpoint_freq=1, + resources=default_resources)] runner.add_trial(trials[0]) runner.step() # start runner.step() self.assertEquals(trials[0].status, Trial.TERMINATED) trials += [ - Trial( - "__fake", - trial_id="trial_fail", - stopping_criterion={"training_iteration": 3}, - checkpoint_freq=1, - checkpoint_at_end=True, - config={"mock_error": True}, - resources=default_resources) - ] + Trial("__fake", + trial_id="trial_fail", + stopping_criterion={"training_iteration": 3}, + checkpoint_freq=1, + config={"mock_error": True}, + resources=default_resources)] runner.add_trial(trials[1]) runner.step() runner.step() @@ -1590,13 +1586,11 @@ def testSaveRestore(self): self.assertEquals(trials[1].status, Trial.ERROR) trials += [ - Trial( - "__fake", - trial_id="trial_succ", - stopping_criterion={"training_iteration": 2}, - checkpoint_freq=1, - resources=default_resources) - ] + Trial("__fake", + trial_id="trial_succ", + stopping_criterion={"training_iteration": 2}, + checkpoint_freq=1, + resources=default_resources)] runner.add_trial(trials[2]) runner.step() self.assertEquals(len(runner.trial_executor.get_checkpoints()), 3) @@ -1618,6 +1612,30 @@ def testSaveRestore(self): self.assertRaises(TuneError, runner2.step) shutil.rmtree(tmpdir) + def testNoSave(self): + """Check that certain trials are not saved.""" + ray.init(num_cpus=3) + tmpdir = tempfile.mkdtemp() + default_resources = Resources(cpu=1, gpu=0) + + runner = TrialRunner( + BasicVariantGenerator(), + checkpoint_dir=tmpdir, + checkpoint_freq=1) + trials = [Trial("__fake", + trial_id="trial_terminate", + stopping_criterion={"training_iteration": 2}, + resources=default_resources)] + runner.add_trial(trials[0]) + runner.step() # start + runner.step() + + runner2 = TrialRunner(BasicVariantGenerator()) + runner2.restore(tmpdir) + self.assertEquals(len(runner2.get_trials()), 0) + runner2.step() + self.assertRaises(TuneError, runner2.step) + shutil.rmtree(tmpdir) if __name__ == "__main__": unittest.main(verbosity=2) diff --git a/python/ray/tune/trial_executor.py b/python/ray/tune/trial_executor.py index 92410906153f..43229d3e704f 100644 --- a/python/ray/tune/trial_executor.py +++ b/python/ray/tune/trial_executor.py @@ -30,11 +30,27 @@ def __init__(self, queue_trials=False, track_checkpoints=False): self._checkpoints = {} def set_status(self, trial, status): + """Sets status and checkpoints metadata if needed. + + Only checkpoints metadata if trial status is a terminal condition. + PENDING, PAUSED, and RUNNING switches have checkpoints taken care of + in the TrialRunner. + + Args: + trial (Trial): Trial to checkpoint. + status (Trial.status): Status to set trial to. + """ trial.status = status - self.checkpoint_metadata_if_needed(trial) + if status in [Trial.TERMINATED, Trial.ERROR]: + self.checkpoint_metadata_if_needed(trial) def checkpoint_metadata_if_needed(self, trial): - if self._track_checkpoints: + """Checkpoints metadata if current session and trial allow. + + Args: + trial (Trial): Trial to checkpoint. + """ + if self._track_checkpoints and trial.checkpoint_freq > 0: if trial._checkpoint.storage == Checkpoint.MEMORY: logger.debug("Not saving data for trial w/ memory checkpoint.") return @@ -46,6 +62,7 @@ def checkpoint_metadata_if_needed(self, trial): logger.exception("Error checkpointing trial metadata.") def get_checkpoints(self): + """Returns a copy of mapping of the trial ID to pickled metadata.""" return self._checkpoints.copy() def has_resources(self, resources): From c13270ba83e79eff076f4ecc7fe8a31df33b12d0 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Sun, 25 Nov 2018 01:27:10 -0800 Subject: [PATCH 58/74] doc --- python/ray/tune/test/trial_runner_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/tune/test/trial_runner_test.py b/python/ray/tune/test/trial_runner_test.py index e47c2e83f208..5ee4ac5f6599 100644 --- a/python/ray/tune/test/trial_runner_test.py +++ b/python/ray/tune/test/trial_runner_test.py @@ -1613,7 +1613,7 @@ def testSaveRestore(self): shutil.rmtree(tmpdir) def testNoSave(self): - """Check that certain trials are not saved.""" + """Check that non-checkpointing trials are not saved.""" ray.init(num_cpus=3) tmpdir = tempfile.mkdtemp() default_resources = Resources(cpu=1, gpu=0) From 8a6ed914ffea78526678e69c3da8114c808b6db8 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Sun, 25 Nov 2018 01:31:17 -0800 Subject: [PATCH 59/74] more docs --- python/ray/tune/ray_trial_executor.py | 1 - python/ray/tune/test/trial_runner_test.py | 1 + python/ray/tune/trial_runner.py | 18 +++++++++++++++--- 3 files changed, 16 insertions(+), 4 deletions(-) diff --git a/python/ray/tune/ray_trial_executor.py b/python/ray/tune/ray_trial_executor.py index 538376a5b5b3..19cff512144c 100644 --- a/python/ray/tune/ray_trial_executor.py +++ b/python/ray/tune/ray_trial_executor.py @@ -5,7 +5,6 @@ import logging import os -import pickle import time import traceback diff --git a/python/ray/tune/test/trial_runner_test.py b/python/ray/tune/test/trial_runner_test.py index 5ee4ac5f6599..08c3df80c5c9 100644 --- a/python/ray/tune/test/trial_runner_test.py +++ b/python/ray/tune/test/trial_runner_test.py @@ -1637,5 +1637,6 @@ def testNoSave(self): self.assertRaises(TuneError, runner2.step) shutil.rmtree(tmpdir) + if __name__ == "__main__": unittest.main(verbosity=2) diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index bec7bee90ae0..b47e8bbc7a5e 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -12,7 +12,7 @@ from ray.tune import TuneError from ray.tune.ray_trial_executor import RayTrialExecutor -from ray.tune.result import TIME_THIS_ITER_S, DEFAULT_RESULTS_DIR +from ray.tune.result import TIME_THIS_ITER_S from ray.tune.trial import Trial, Checkpoint from ray.tune.schedulers import FIFOScheduler, TrialScheduler from ray.tune.web_server import TuneServer @@ -68,7 +68,7 @@ def __init__(self, scheduler (TrialScheduler): Defaults to FIFOScheduler. launch_web_server (bool): Flag for starting TuneServer checkpoint_dir (str): Path where global checkpoints are stored. - checkpoint_freq (int): How many trial results between global + checkpoint_freq (int): How many steps between global checkpoints. A value of 0 (default) disables checkpointing. server_port (int): Port number for launching TuneServer verbose (bool): Flag for verbosity. If False, trial results @@ -103,7 +103,7 @@ def __init__(self, self._trial_checkpoints = {} def save(self): - """Saves all trial checkpoints""" + """Saves all trial checkpoints to `self._checkpoint_dir.`""" checkpoint_dir = self._checkpoint_dir if not os.path.exists(checkpoint_dir): logger.debug("Checkpoint directory newly created.") @@ -123,6 +123,14 @@ def save(self): return checkpoint_dir def restore(self, checkpoint_dir): + """Restores all checkpointed trials from previous run. + + Requires user to manually re-register their objects. Also stops + all ongoing trials. + + Args: + checkpoint_dir (str): Path to checkpoint (previously specified). + """ logger.debug("Stopping all trials.") for trial in self._trials: self.stop_trial(trial) @@ -371,6 +379,10 @@ def try_recover(self, trial, error_msg): """Tries to recover trial. Notifies SearchAlgorithm and Scheduler if failure to recover. + + Args: + trial (Trial): Trial to recover. + error_msg (str): Error message from prior to invoking this method. """ try: self.trial_executor.stop_trial( From b1e3bf0faa328af3cad7324dbfc43879fa78e921 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Sun, 25 Nov 2018 01:36:30 -0800 Subject: [PATCH 60/74] test docs --- python/ray/tune/test/cluster_tests.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 5af6a5503aec..f6d6711ec255 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -294,7 +294,7 @@ def test_trial_requeue(start_connected_emptyhead_cluster): def test_cluster_down_simple(start_connected_cluster): - """Removing a node in full cluster causes Trial to be requeued.""" + """Tests that TrialRunner save/restore works on cluster shutdown.""" cluster = start_connected_cluster cluster.add_node(resources=dict(CPU=1)) tmpdir = tempfile.mkdtemp() @@ -338,6 +338,7 @@ def test_cluster_down_simple(start_connected_cluster): def test_cluster_down_full(start_connected_cluster, tmpdir): + """Tests that run_experiment restoring works on cluster shutdown.""" cluster = start_connected_cluster script = """ @@ -387,6 +388,7 @@ def test_cluster_down_full(start_connected_cluster, tmpdir): def test_cluster_down_error(start_connected_cluster, tmpdir): + """Tests run_experiment on cluster shutdown even with atypical trial.""" cluster = start_connected_cluster script = """ From 40248aa111fece1b31ed67e35623ff50bba991fa Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Sun, 25 Nov 2018 18:01:19 -0800 Subject: [PATCH 61/74] py2mock --- .travis/install-dependencies.sh | 4 ++-- python/ray/tune/test/trial_runner_test.py | 12 ++++++++---- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/.travis/install-dependencies.sh b/.travis/install-dependencies.sh index 293c1b8b6b04..3f1ea4922bc8 100755 --- a/.travis/install-dependencies.sh +++ b/.travis/install-dependencies.sh @@ -25,7 +25,7 @@ if [[ "$PYTHON" == "2.7" ]] && [[ "$platform" == "linux" ]]; then bash miniconda.sh -b -p $HOME/miniconda export PATH="$HOME/miniconda/bin:$PATH" pip install -q cython==0.27.3 cmake tensorflow gym opencv-python pyyaml pandas==0.22 requests \ - feather-format lxml openpyxl xlrd py-spy setproctitle faulthandler pytest-timeout + feather-format lxml openpyxl xlrd py-spy setproctitle faulthandler pytest-timeout mock elif [[ "$PYTHON" == "3.5" ]] && [[ "$platform" == "linux" ]]; then sudo apt-get update sudo apt-get install -y cmake pkg-config python-dev python-numpy build-essential autoconf curl libtool unzip @@ -51,7 +51,7 @@ elif [[ "$PYTHON" == "2.7" ]] && [[ "$platform" == "macosx" ]]; then bash miniconda.sh -b -p $HOME/miniconda export PATH="$HOME/miniconda/bin:$PATH" pip install -q cython==0.27.3 cmake tensorflow gym opencv-python pyyaml pandas==0.22 requests \ - feather-format lxml openpyxl xlrd py-spy setproctitle faulthandler pytest-timeout + feather-format lxml openpyxl xlrd py-spy setproctitle faulthandler pytest-timeout mock elif [[ "$PYTHON" == "3.5" ]] && [[ "$platform" == "macosx" ]]; then # check that brew is installed which -s brew diff --git a/python/ray/tune/test/trial_runner_test.py b/python/ray/tune/test/trial_runner_test.py index 08c3df80c5c9..c89037d07483 100644 --- a/python/ray/tune/test/trial_runner_test.py +++ b/python/ray/tune/test/trial_runner_test.py @@ -4,10 +4,14 @@ import os import shutil +import sys +if sys.version_info >= (3, 3): + from unittest.mock import patch +else: + from mock import patch import tempfile import time import unittest -from unittest import mock import ray from ray.rllib import _register_all @@ -1137,15 +1141,15 @@ def testFailureRecoveryNodeRemoval(self): runner.add_trial(Trial("__fake", **kwargs)) trials = runner.get_trials() - with mock.patch('ray.global_state.cluster_resources') as res_mock: - res_mock.return_value = {"CPU": 1, "GPU": 1} + with patch('ray.global_state.cluster_resources') as resource_mock: + resource_mock.return_value = {"CPU": 1, "GPU": 1} runner.step() self.assertEqual(trials[0].status, Trial.RUNNING) runner.step() self.assertEqual(trials[0].status, Trial.RUNNING) # Mimic a node failure - res_mock.return_value = {"CPU": 0, "GPU": 0} + resource_mock.return_value = {"CPU": 0, "GPU": 0} runner.step() self.assertEqual(trials[0].status, Trial.PENDING) self.assertEqual(trials[0].num_failures, 1) From 22930c80d08d5ce8efd50d1334c06ba48b9aead5 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 00:22:41 -0800 Subject: [PATCH 62/74] dirpath from tmpdir --- python/ray/tune/test/cluster_tests.py | 28 +++++++++++++-------------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index f6d6711ec255..3d948394cdde 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -293,12 +293,12 @@ def test_trial_requeue(start_connected_emptyhead_cluster): runner.step() -def test_cluster_down_simple(start_connected_cluster): +def test_cluster_down_simple(start_connected_cluster, tmpdir): """Tests that TrialRunner save/restore works on cluster shutdown.""" cluster = start_connected_cluster cluster.add_node(resources=dict(CPU=1)) - tmpdir = tempfile.mkdtemp() - runner = TrialRunner(BasicVariantGenerator(), checkpoint_dir=tmpdir) + dirpath = str(tmpdir) + runner = TrialRunner(BasicVariantGenerator(), checkpoint_dir=dirpath) kwargs = { "stopping_criterion": { "training_iteration": 2 @@ -323,7 +323,7 @@ def test_cluster_down_simple(start_connected_cluster): cluster = _start_new_cluster() register_test_trainable() runner = TrialRunner(BasicVariantGenerator()) - runner.restore(tmpdir) + runner.restore(dirpath) print([t.status for t in runner.get_trials()]) runner.step() # start runner.step() # start2 @@ -340,7 +340,7 @@ def test_cluster_down_simple(start_connected_cluster): def test_cluster_down_full(start_connected_cluster, tmpdir): """Tests that run_experiment restoring works on cluster shutdown.""" cluster = start_connected_cluster - + dirpath = str(tmpdir) script = """ import os import time @@ -365,12 +365,12 @@ def test_cluster_down_full(start_connected_cluster, tmpdir): checkpoint_freq=3) """.format( redis_address=cluster.redis_address, - checkpoint_dir=tmpdir, + checkpoint_dir=dirpath, register_trainable_fn=inspect.getsource(register_test_trainable), run_register_trainable_fn=register_test_trainable.__name__) run_string_as_driver_nonblocking(script) - while not os.path.exists(os.path.join(tmpdir, "experiment.state")): + while not os.path.exists(os.path.join(dirpath, "experiment.state")): time.sleep(0.1) ray.shutdown() cluster.shutdown() @@ -379,18 +379,18 @@ def test_cluster_down_full(start_connected_cluster, tmpdir): # Check that last_result.iteration = 1 runner = TrialRunner(BasicVariantGenerator()) - runner.restore(tmpdir) + runner.restore(dirpath) trials = runner.get_trials() assert trials[0].last_result["training_iteration"] == 1 - trials = tune.run_experiments(restore_from_path=tmpdir) + trials = tune.run_experiments(restore_from_path=dirpath) assert all(t.status == Trial.TERMINATED for t in trials) def test_cluster_down_error(start_connected_cluster, tmpdir): """Tests run_experiment on cluster shutdown even with atypical trial.""" cluster = start_connected_cluster - + dirpath = str(tmpdir) script = """ import os import time @@ -415,12 +415,12 @@ def test_cluster_down_error(start_connected_cluster, tmpdir): checkpoint_freq=3) """.format( redis_address=cluster.redis_address, - checkpoint_dir=tmpdir, + checkpoint_dir=dirpath, register_trainable_fn=inspect.getsource(register_fail_trainable), run_register_trainable_fn=register_fail_trainable.__name__) run_string_as_driver_nonblocking(script) - while not os.path.exists(os.path.join(tmpdir, "experiment.state")): + while not os.path.exists(os.path.join(dirpath, "experiment.state")): time.sleep(0.1) ray.shutdown() cluster.shutdown() @@ -430,12 +430,12 @@ def test_cluster_down_error(start_connected_cluster, tmpdir): # Inspect the internal trialrunner runner = TrialRunner(BasicVariantGenerator()) - runner.restore(tmpdir) + runner.restore(dirpath) trials = runner.get_trials() assert trials[0].last_result["training_iteration"] == 1 assert trials[0].status == Trial.PENDING # Restore properly from checkpoint trials = tune.run_experiments( - restore_from_path=tmpdir, raise_on_failed_trial=False) + restore_from_path=dirpath, raise_on_failed_trial=False) assert all(t.status == Trial.ERROR for t in trials) From 82ff45e38685cffea530de4c0cf501af5a13fd57 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 01:17:15 -0800 Subject: [PATCH 63/74] fix tsts? --- python/ray/tune/test/cluster_tests.py | 42 +++++++++++++-------------- 1 file changed, 21 insertions(+), 21 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 3d948394cdde..34f48b3d18fa 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -16,7 +16,7 @@ import ray from ray import tune from ray.test.cluster_utils import Cluster -from ray.test.test_utils import run_string_as_driver_nonblocking +from ray.test.test_utils import run_string_as_driver from ray.tune.error import TuneError from ray.tune.trial import Trial from ray.tune.trial_runner import TrialRunner @@ -43,14 +43,15 @@ def _restore(self, state): def register_fail_trainable(): - class _Train(tune.Trainable): + class _Fail(tune.Trainable): + """Fails on the 4th iteration.""" def _setup(self, config): - self.state = {"hi": 1} + self.state = {"hi": 0} def _train(self): self.state["hi"] += 1 time.sleep(0.5) - if self.state["hi"] % 2 == 1: + if self.state["hi"] % 5 == 4: assert False return {} @@ -60,7 +61,7 @@ def _save(self, path): def _restore(self, state): self.state = state - tune.register_trainable("fail", _Train) + tune.register_trainable("test2", _Fail) def _start_new_cluster(): @@ -298,7 +299,10 @@ def test_cluster_down_simple(start_connected_cluster, tmpdir): cluster = start_connected_cluster cluster.add_node(resources=dict(CPU=1)) dirpath = str(tmpdir) - runner = TrialRunner(BasicVariantGenerator(), checkpoint_dir=dirpath) + runner = TrialRunner( + BasicVariantGenerator(), + checkpoint_freq=2, + checkpoint_dir=dirpath) kwargs = { "stopping_criterion": { "training_iteration": 2 @@ -335,6 +339,7 @@ def test_cluster_down_simple(start_connected_cluster, tmpdir): runner.step() assert all(t.status == Trial.TERMINATED for t in runner.get_trials()) + cluster.shutdown() def test_cluster_down_full(start_connected_cluster, tmpdir): @@ -362,16 +367,13 @@ def test_cluster_down_full(start_connected_cluster, tmpdir): tune.run_experiments( dict(experiment=kwargs), checkpoint_dir="{checkpoint_dir}", - checkpoint_freq=3) + checkpoint_freq=2) # start, iter 1 """.format( redis_address=cluster.redis_address, checkpoint_dir=dirpath, register_trainable_fn=inspect.getsource(register_test_trainable), run_register_trainable_fn=register_test_trainable.__name__) - run_string_as_driver_nonblocking(script) - - while not os.path.exists(os.path.join(dirpath, "experiment.state")): - time.sleep(0.1) + run_string_as_driver(script) ray.shutdown() cluster.shutdown() cluster = _start_new_cluster() @@ -381,10 +383,11 @@ def test_cluster_down_full(start_connected_cluster, tmpdir): runner = TrialRunner(BasicVariantGenerator()) runner.restore(dirpath) trials = runner.get_trials() - assert trials[0].last_result["training_iteration"] == 1 + assert trials[0].last_result["training_iteration"] == 2 trials = tune.run_experiments(restore_from_path=dirpath) assert all(t.status == Trial.TERMINATED for t in trials) + cluster.shutdown() def test_cluster_down_error(start_connected_cluster, tmpdir): @@ -397,14 +400,13 @@ def test_cluster_down_error(start_connected_cluster, tmpdir): import ray from ray import tune - ray.init(redis_address="{redis_address}") {register_trainable_fn} {run_register_trainable_fn}() kwargs = dict( - run="fail", + run="test2", stop=dict(training_iteration=5), checkpoint_freq=1, max_failures=1) @@ -412,19 +414,16 @@ def test_cluster_down_error(start_connected_cluster, tmpdir): tune.run_experiments( dict(experiment1=kwargs), checkpoint_dir="{checkpoint_dir}", - checkpoint_freq=3) + checkpoint_freq=3, + raise_on_failed_trial=False) """.format( redis_address=cluster.redis_address, checkpoint_dir=dirpath, register_trainable_fn=inspect.getsource(register_fail_trainable), run_register_trainable_fn=register_fail_trainable.__name__) - run_string_as_driver_nonblocking(script) - - while not os.path.exists(os.path.join(dirpath, "experiment.state")): - time.sleep(0.1) + run_string_as_driver(script) ray.shutdown() cluster.shutdown() - cluster = _start_new_cluster() register_fail_trainable() @@ -432,10 +431,11 @@ def test_cluster_down_error(start_connected_cluster, tmpdir): runner = TrialRunner(BasicVariantGenerator()) runner.restore(dirpath) trials = runner.get_trials() - assert trials[0].last_result["training_iteration"] == 1 + assert trials[0].last_result["training_iteration"] == 3 assert trials[0].status == Trial.PENDING # Restore properly from checkpoint trials = tune.run_experiments( restore_from_path=dirpath, raise_on_failed_trial=False) assert all(t.status == Trial.ERROR for t in trials) + cluster.shutdown() From bde644c62f57a18e82e10cb3c4e135d0af804049 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 01:25:54 -0800 Subject: [PATCH 64/74] yapf --- python/ray/tune/test/cluster_tests.py | 11 +--- python/ray/tune/test/trial_runner_test.py | 68 ++++++++++++----------- 2 files changed, 40 insertions(+), 39 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 34f48b3d18fa..5e747d9ef9ba 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -4,9 +4,7 @@ import inspect import json -import os import time -import tempfile import pytest try: import pytest_timeout @@ -45,13 +43,14 @@ def _restore(self, state): def register_fail_trainable(): class _Fail(tune.Trainable): """Fails on the 4th iteration.""" + def _setup(self, config): self.state = {"hi": 0} def _train(self): self.state["hi"] += 1 time.sleep(0.5) - if self.state["hi"] % 5 == 4: + if self.state["hi"] >= 4: assert False return {} @@ -300,9 +299,7 @@ def test_cluster_down_simple(start_connected_cluster, tmpdir): cluster.add_node(resources=dict(CPU=1)) dirpath = str(tmpdir) runner = TrialRunner( - BasicVariantGenerator(), - checkpoint_freq=2, - checkpoint_dir=dirpath) + BasicVariantGenerator(), checkpoint_freq=2, checkpoint_dir=dirpath) kwargs = { "stopping_criterion": { "training_iteration": 2 @@ -347,7 +344,6 @@ def test_cluster_down_full(start_connected_cluster, tmpdir): cluster = start_connected_cluster dirpath = str(tmpdir) script = """ -import os import time import ray from ray import tune @@ -395,7 +391,6 @@ def test_cluster_down_error(start_connected_cluster, tmpdir): cluster = start_connected_cluster dirpath = str(tmpdir) script = """ -import os import time import ray from ray import tune diff --git a/python/ray/tune/test/trial_runner_test.py b/python/ray/tune/test/trial_runner_test.py index c89037d07483..1015e3376d0a 100644 --- a/python/ray/tune/test/trial_runner_test.py +++ b/python/ray/tune/test/trial_runner_test.py @@ -5,17 +5,12 @@ import os import shutil import sys -if sys.version_info >= (3, 3): - from unittest.mock import patch -else: - from mock import patch import tempfile import time import unittest import ray from ray.rllib import _register_all - from ray.tune import Trainable, TuneError from ray.tune import register_env, register_trainable, run_experiments from ray.tune.ray_trial_executor import RayTrialExecutor @@ -32,6 +27,11 @@ SuggestionAlgorithm) from ray.tune.suggest.variant_generator import RecursiveDependencyError +if sys.version_info >= (3, 3): + from unittest.mock import patch +else: + from mock import patch + class TrainableFunctionApiTest(unittest.TestCase): def setUp(self): @@ -1563,26 +1563,29 @@ def testSaveRestore(self): default_resources = Resources(cpu=1, gpu=0) runner = TrialRunner( - BasicVariantGenerator(), - checkpoint_dir=tmpdir, - checkpoint_freq=1) - trials = [Trial("__fake", - trial_id="trial_terminate", - stopping_criterion={"training_iteration": 1}, - checkpoint_freq=1, - resources=default_resources)] + BasicVariantGenerator(), checkpoint_dir=tmpdir, checkpoint_freq=1) + trials = [ + Trial( + "__fake", + trial_id="trial_terminate", + stopping_criterion={"training_iteration": 1}, + checkpoint_freq=1, + resources=default_resources) + ] runner.add_trial(trials[0]) runner.step() # start runner.step() self.assertEquals(trials[0].status, Trial.TERMINATED) trials += [ - Trial("__fake", - trial_id="trial_fail", - stopping_criterion={"training_iteration": 3}, - checkpoint_freq=1, - config={"mock_error": True}, - resources=default_resources)] + Trial( + "__fake", + trial_id="trial_fail", + stopping_criterion={"training_iteration": 3}, + checkpoint_freq=1, + config={"mock_error": True}, + resources=default_resources) + ] runner.add_trial(trials[1]) runner.step() runner.step() @@ -1590,11 +1593,13 @@ def testSaveRestore(self): self.assertEquals(trials[1].status, Trial.ERROR) trials += [ - Trial("__fake", - trial_id="trial_succ", - stopping_criterion={"training_iteration": 2}, - checkpoint_freq=1, - resources=default_resources)] + Trial( + "__fake", + trial_id="trial_succ", + stopping_criterion={"training_iteration": 2}, + checkpoint_freq=1, + resources=default_resources) + ] runner.add_trial(trials[2]) runner.step() self.assertEquals(len(runner.trial_executor.get_checkpoints()), 3) @@ -1623,13 +1628,14 @@ def testNoSave(self): default_resources = Resources(cpu=1, gpu=0) runner = TrialRunner( - BasicVariantGenerator(), - checkpoint_dir=tmpdir, - checkpoint_freq=1) - trials = [Trial("__fake", - trial_id="trial_terminate", - stopping_criterion={"training_iteration": 2}, - resources=default_resources)] + BasicVariantGenerator(), checkpoint_dir=tmpdir, checkpoint_freq=1) + trials = [ + Trial( + "__fake", + trial_id="trial_terminate", + stopping_criterion={"training_iteration": 2}, + resources=default_resources) + ] runner.add_trial(trials[0]) runner.step() # start runner.step() From 79197b8d9f27cfbb425cc78515cdde3317e2f6a5 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 11:44:48 -0800 Subject: [PATCH 65/74] Fix up tests --- python/ray/tune/test/cluster_tests.py | 107 +++++++++++++++----------- python/ray/tune/trial_executor.py | 4 +- python/ray/tune/trial_runner.py | 5 +- 3 files changed, 65 insertions(+), 51 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 5e747d9ef9ba..558f88d4ec54 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -5,6 +5,7 @@ import inspect import json import time +import os import pytest try: import pytest_timeout @@ -14,32 +15,13 @@ import ray from ray import tune from ray.test.cluster_utils import Cluster -from ray.test.test_utils import run_string_as_driver +from ray.test.test_utils import run_string_as_driver, run_string_as_driver_nonblocking from ray.tune.error import TuneError from ray.tune.trial import Trial from ray.tune.trial_runner import TrialRunner from ray.tune.suggest import BasicVariantGenerator -def register_test_trainable(): - class _Train(tune.Trainable): - def _setup(self, config): - self.state = {"hi": 1} - - def _train(self): - self.state["hi"] += 1 - time.sleep(0.5) - return {} - - def _save(self, path): - return self.state - - def _restore(self, state): - self.state = state - - tune.register_trainable("test", _Train) - - def register_fail_trainable(): class _Fail(tune.Trainable): """Fails on the 4th iteration.""" @@ -79,7 +61,6 @@ def _start_new_cluster(): def start_connected_cluster(): # Start the Ray processes. cluster = _start_new_cluster() - register_test_trainable() yield cluster # The code after the yield will run as teardown code. ray.shutdown() @@ -99,7 +80,6 @@ def start_connected_emptyhead_cluster(): "num_heartbeats_timeout": 10 }) }) - register_test_trainable() yield cluster # The code after the yield will run as teardown code. ray.shutdown() @@ -123,7 +103,7 @@ def test_counting_resources(start_connected_cluster): runner = TrialRunner(BasicVariantGenerator()) kwargs = {"stopping_criterion": {"training_iteration": 10}} - trials = [Trial("test", **kwargs), Trial("test", **kwargs)] + trials = [Trial("__fake", **kwargs), Trial("__fake", **kwargs)] for t in trials: runner.add_trial(t) @@ -161,7 +141,7 @@ def test_remove_node_before_result(start_connected_cluster): runner = TrialRunner(BasicVariantGenerator()) kwargs = {"stopping_criterion": {"training_iteration": 3}} - trials = [Trial("test", **kwargs), Trial("test", **kwargs)] + trials = [Trial("__fake", **kwargs), Trial("__fake", **kwargs)] for t in trials: runner.add_trial(t) @@ -207,7 +187,7 @@ def test_trial_migration(start_connected_emptyhead_cluster): } # Test recovery of trial that hasn't been checkpointed - t = Trial("test", **kwargs) + t = Trial("__fake", **kwargs) runner.add_trial(t) runner.step() # start runner.step() # 1 result @@ -227,7 +207,7 @@ def test_trial_migration(start_connected_emptyhead_cluster): assert t.status == Trial.TERMINATED # Test recovery of trial that has been checkpointed - t2 = Trial("test", **kwargs) + t2 = Trial("__fake", **kwargs) runner.add_trial(t2) runner.step() # start runner.step() # 1 result @@ -244,7 +224,7 @@ def test_trial_migration(start_connected_emptyhead_cluster): assert t2.status == Trial.TERMINATED # Test recovery of trial that won't be checkpointed - t3 = Trial("test", **{"stopping_criterion": {"training_iteration": 3}}) + t3 = Trial("__fake", **{"stopping_criterion": {"training_iteration": 3}}) runner.add_trial(t3) runner.step() # start runner.step() # 1 result @@ -277,7 +257,7 @@ def test_trial_requeue(start_connected_emptyhead_cluster): "max_failures": 1 } - trials = [Trial("test", **kwargs), Trial("test", **kwargs)] + trials = [Trial("__fake", **kwargs), Trial("__fake", **kwargs)] for t in trials: runner.add_trial(t) @@ -307,8 +287,7 @@ def test_cluster_down_simple(start_connected_cluster, tmpdir): "checkpoint_freq": 1, "max_failures": 1 } - register_test_trainable() - trials = [Trial("test", **kwargs), Trial("test", **kwargs)] + trials = [Trial("__fake", **kwargs), Trial("__fake", **kwargs)] for t in trials: runner.add_trial(t) @@ -351,43 +330,62 @@ def test_cluster_down_full(start_connected_cluster, tmpdir): ray.init(redis_address="{redis_address}") -{register_trainable_fn} -{run_register_trainable_fn}() +exp1_args = dict( + run="__fake", + stop=dict(training_iteration=3), + checkpoint_freq=1, + max_failures=1) -kwargs = dict( - run="test", +exp2_args = dict( + run="__fake", + stop=dict(training_iteration=3)) + +exp3_args = dict( + run="__fake", + stop=dict(training_iteration=3), + config=dict(mock_error=True)) + +exp4_args = dict( + run="__fake", stop=dict(training_iteration=3), + config=dict(mock_error=True), checkpoint_freq=1, max_failures=1) tune.run_experiments( - dict(experiment=kwargs), + {"exp1": exp1_args, + "exp2": exp2_args, + "exp3": exp3_args, + "exp4": exp4_args, + ), checkpoint_dir="{checkpoint_dir}", - checkpoint_freq=2) # start, iter 1 + checkpoint_freq=2) """.format( - redis_address=cluster.redis_address, - checkpoint_dir=dirpath, - register_trainable_fn=inspect.getsource(register_test_trainable), - run_register_trainable_fn=register_test_trainable.__name__) + redis_address=cluster.redis_address, checkpoint_dir=dirpath) run_string_as_driver(script) ray.shutdown() cluster.shutdown() cluster = _start_new_cluster() - register_test_trainable() # Check that last_result.iteration = 1 runner = TrialRunner(BasicVariantGenerator()) runner.restore(dirpath) trials = runner.get_trials() - assert trials[0].last_result["training_iteration"] == 2 + assert len(trials) == 1 + assert trials[0].last_result["training_iteration"] == 3 trials = tune.run_experiments(restore_from_path=dirpath) - assert all(t.status == Trial.TERMINATED for t in trials) + assert len(trials) == 2 + assert all(t.status in [Trial.TERMINATED, Trial.ERROR] for t in trials) cluster.shutdown() -def test_cluster_down_error(start_connected_cluster, tmpdir): - """Tests run_experiment on cluster shutdown even with atypical trial.""" +def test_cluster_interrupt(start_connected_cluster, tmpdir): + """Tests run_experiment on cluster shutdown even with atypical trial. + + The trial fails on the 4th step, and the checkpointing happens on + the 3rd step, so restoring should actually launch the trial again. + """ cluster = start_connected_cluster dirpath = str(tmpdir) script = """ @@ -406,6 +404,7 @@ def test_cluster_down_error(start_connected_cluster, tmpdir): checkpoint_freq=1, max_failures=1) +# This will save to disk on step 0 and step 3 tune.run_experiments( dict(experiment1=kwargs), checkpoint_dir="{checkpoint_dir}", @@ -416,13 +415,27 @@ def test_cluster_down_error(start_connected_cluster, tmpdir): checkpoint_dir=dirpath, register_trainable_fn=inspect.getsource(register_fail_trainable), run_register_trainable_fn=register_fail_trainable.__name__) - run_string_as_driver(script) + run_string_as_driver_nonblocking(script) + + # Wait until the right checkpoint is saved. + # The trainable returns every 0.5 seconds, so this should not miss + # the checkpoint. + for i in range(30): + if os.path.exists(os.path.join(dirpath, "experiment.state")): + # Inspect the internal trialrunner + runner = TrialRunner(BasicVariantGenerator()) + runner.restore(dirpath) + trials = runner.get_trials() + if trials[0].last_result["training_iteration"] == 3: + break + time.sleep(0.2) + ray.shutdown() cluster.shutdown() cluster = _start_new_cluster() register_fail_trainable() - # Inspect the internal trialrunner + # Inspect the internal trialrunner just in case runner = TrialRunner(BasicVariantGenerator()) runner.restore(dirpath) trials = runner.get_trials() diff --git a/python/ray/tune/trial_executor.py b/python/ray/tune/trial_executor.py index 43229d3e704f..53b9d1d805c2 100644 --- a/python/ray/tune/trial_executor.py +++ b/python/ray/tune/trial_executor.py @@ -42,9 +42,9 @@ def set_status(self, trial, status): """ trial.status = status if status in [Trial.TERMINATED, Trial.ERROR]: - self.checkpoint_metadata_if_needed(trial) + self.try_checkpoint_metadata(trial) - def checkpoint_metadata_if_needed(self, trial): + def try_checkpoint_metadata(self, trial): """Checkpoints metadata if current session and trial allow. Args: diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index b47e8bbc7a5e..548c68c51cc0 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -192,7 +192,8 @@ def step(self): "trials with sufficient resources.") if self._checkpoint_freq: - if self._iteration % self._checkpoint_freq == 0: + if (self._iteration % self._checkpoint_freq == 0 + or self.is_finished()): self.save() self._iteration += 1 @@ -373,7 +374,7 @@ def _checkpoint_if_needed(self, trial): # Save trial runtime if possible if hasattr(trial, "runner") and trial.runner: self.trial_executor.save(trial, storage=Checkpoint.DISK) - self.trial_executor.checkpoint_metadata_if_needed(trial) + self.trial_executor.try_checkpoint_metadata(trial) def try_recover(self, trial, error_msg): """Tries to recover trial. From 66be742cf38610246a6820e26c7f33f6b1820f0b Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 11:47:44 -0800 Subject: [PATCH 66/74] nits --- python/ray/tune/test/cluster_tests.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 558f88d4ec54..0b01a863093c 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -15,7 +15,8 @@ import ray from ray import tune from ray.test.cluster_utils import Cluster -from ray.test.test_utils import run_string_as_driver, run_string_as_driver_nonblocking +from ray.test.test_utils import ( + run_string_as_driver, run_string_as_driver_nonblocking) from ray.tune.error import TuneError from ray.tune.trial import Trial from ray.tune.trial_runner import TrialRunner @@ -301,7 +302,6 @@ def test_cluster_down_simple(start_connected_cluster, tmpdir): ray.shutdown() cluster = _start_new_cluster() - register_test_trainable() runner = TrialRunner(BasicVariantGenerator()) runner.restore(dirpath) print([t.status for t in runner.get_trials()]) From 25be8434f0b6c40281bbc20fb11b3904c532f956 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 11:58:27 -0800 Subject: [PATCH 67/74] nit --- python/ray/tune/test/cluster_tests.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 0b01a863093c..0c978d5b11d6 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -93,7 +93,6 @@ def start_connected_emptyhead_cluster(): @pytest.mark.timeout(10, method="thread") def test_counting_resources(start_connected_cluster): """Tests that Tune accounting is consistent with actual cluster.""" - cluster = start_connected_cluster assert ray.global_state.cluster_resources()["CPU"] == 1 nodes = [] @@ -356,8 +355,7 @@ def test_cluster_down_full(start_connected_cluster, tmpdir): {"exp1": exp1_args, "exp2": exp2_args, "exp3": exp3_args, - "exp4": exp4_args, - ), + "exp4": exp4_args}, checkpoint_dir="{checkpoint_dir}", checkpoint_freq=2) """.format( From ff7b1141127ad038afe26d133e389b04cc1058bd Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 12:55:55 -0800 Subject: [PATCH 68/74] test fixup --- python/ray/tune/test/cluster_tests.py | 85 ++++++++++++--------------- 1 file changed, 37 insertions(+), 48 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 0c978d5b11d6..488178995a76 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -14,6 +14,7 @@ import ray from ray import tune +from ray.rllib import _register_all from ray.test.cluster_utils import Cluster from ray.test.test_utils import ( run_string_as_driver, run_string_as_driver_nonblocking) @@ -47,7 +48,7 @@ def _restore(self, state): def _start_new_cluster(): - return Cluster( + cluster = Cluster( initialize_head=True, connect=True, head_node_args={ @@ -56,6 +57,9 @@ def _start_new_cluster(): "num_heartbeats_timeout": 10 }) }) + # Pytest doesn't play nicely with imports + _register_all() + return cluster @pytest.fixture @@ -81,16 +85,15 @@ def start_connected_emptyhead_cluster(): "num_heartbeats_timeout": 10 }) }) + # Pytest doesn't play nicely with imports + _register_all() yield cluster # The code after the yield will run as teardown code. ray.shutdown() cluster.shutdown() -@pytest.mark.skipif( - pytest_timeout is None, - reason="Timeout package not installed; skipping test.") -@pytest.mark.timeout(10, method="thread") +@pytest.mark.skip("Add this test once reconstruction is fixed") def test_counting_resources(start_connected_cluster): """Tests that Tune accounting is consistent with actual cluster.""" cluster = start_connected_cluster @@ -321,46 +324,34 @@ def test_cluster_down_full(start_connected_cluster, tmpdir): """Tests that run_experiment restoring works on cluster shutdown.""" cluster = start_connected_cluster dirpath = str(tmpdir) - script = """ -import time -import ray -from ray import tune - - -ray.init(redis_address="{redis_address}") - -exp1_args = dict( - run="__fake", - stop=dict(training_iteration=3), - checkpoint_freq=1, - max_failures=1) - -exp2_args = dict( - run="__fake", - stop=dict(training_iteration=3)) - -exp3_args = dict( - run="__fake", - stop=dict(training_iteration=3), - config=dict(mock_error=True)) - -exp4_args = dict( - run="__fake", - stop=dict(training_iteration=3), - config=dict(mock_error=True), - checkpoint_freq=1, - max_failures=1) + exp1_args = dict( + run="__fake", + stop=dict(training_iteration=3), + checkpoint_freq=1, + max_failures=1) + exp2_args = dict( + run="__fake", + stop=dict(training_iteration=3)) + exp3_args = dict( + run="__fake", + stop=dict(training_iteration=3), + config=dict(mock_error=True)) + exp4_args = dict( + run="__fake", + stop=dict(training_iteration=3), + config=dict(mock_error=True), + checkpoint_freq=1, + max_failures=1) + + tune.run_experiments( + dict(exp1=exp1_args, + exp2=exp2_args, + exp3=exp3_args, + exp4=exp4_args), + checkpoint_dir=dirpath, + checkpoint_freq=2, + raise_on_failed_trial=False) -tune.run_experiments( - {"exp1": exp1_args, - "exp2": exp2_args, - "exp3": exp3_args, - "exp4": exp4_args}, - checkpoint_dir="{checkpoint_dir}", - checkpoint_freq=2) -""".format( - redis_address=cluster.redis_address, checkpoint_dir=dirpath) - run_string_as_driver(script) ray.shutdown() cluster.shutdown() cluster = _start_new_cluster() @@ -369,9 +360,6 @@ def test_cluster_down_full(start_connected_cluster, tmpdir): runner = TrialRunner(BasicVariantGenerator()) runner.restore(dirpath) trials = runner.get_trials() - assert len(trials) == 1 - assert trials[0].last_result["training_iteration"] == 3 - trials = tune.run_experiments(restore_from_path=dirpath) assert len(trials) == 2 assert all(t.status in [Trial.TERMINATED, Trial.ERROR] for t in trials) @@ -424,7 +412,8 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): runner = TrialRunner(BasicVariantGenerator()) runner.restore(dirpath) trials = runner.get_trials() - if trials[0].last_result["training_iteration"] == 3: + last_res = trials[0].last_result + if last_res is not None and last_res["training_iteration"] == 3: break time.sleep(0.2) From defe5247e401e2f2acda2d27b30716f9ef1f924d Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 12:57:51 -0800 Subject: [PATCH 69/74] yapf --- python/ray/tune/test/cluster_tests.py | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 488178995a76..fc9d670a03e6 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -16,8 +16,7 @@ from ray import tune from ray.rllib import _register_all from ray.test.cluster_utils import Cluster -from ray.test.test_utils import ( - run_string_as_driver, run_string_as_driver_nonblocking) +from ray.test.test_utils import run_string_as_driver_nonblocking from ray.tune.error import TuneError from ray.tune.trial import Trial from ray.tune.trial_runner import TrialRunner @@ -329,9 +328,7 @@ def test_cluster_down_full(start_connected_cluster, tmpdir): stop=dict(training_iteration=3), checkpoint_freq=1, max_failures=1) - exp2_args = dict( - run="__fake", - stop=dict(training_iteration=3)) + exp2_args = dict(run="__fake", stop=dict(training_iteration=3)) exp3_args = dict( run="__fake", stop=dict(training_iteration=3), @@ -342,12 +339,9 @@ def test_cluster_down_full(start_connected_cluster, tmpdir): config=dict(mock_error=True), checkpoint_freq=1, max_failures=1) - + tune.run_experiments( - dict(exp1=exp1_args, - exp2=exp2_args, - exp3=exp3_args, - exp4=exp4_args), + dict(exp1=exp1_args, exp2=exp2_args, exp3=exp3_args, exp4=exp4_args), checkpoint_dir=dirpath, checkpoint_freq=2, raise_on_failed_trial=False) From 02a9cf807f687e9c11900e49220235adf4fd9f59 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 13:05:05 -0800 Subject: [PATCH 70/74] no skip --- python/ray/tune/test/cluster_tests.py | 1 - 1 file changed, 1 deletion(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index fc9d670a03e6..97eaf6b7411d 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -92,7 +92,6 @@ def start_connected_emptyhead_cluster(): cluster.shutdown() -@pytest.mark.skip("Add this test once reconstruction is fixed") def test_counting_resources(start_connected_cluster): """Tests that Tune accounting is consistent with actual cluster.""" cluster = start_connected_cluster From f80e318e0df618e2abb48d4184dec119f0f88f3c Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 13:26:05 -0800 Subject: [PATCH 71/74] cluster tests --- python/ray/tune/test/cluster_tests.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 97eaf6b7411d..66e732d18d52 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -95,6 +95,7 @@ def start_connected_emptyhead_cluster(): def test_counting_resources(start_connected_cluster): """Tests that Tune accounting is consistent with actual cluster.""" cluster = start_connected_cluster + register_fail_trainable() # This is less flaky than _register_all assert ray.global_state.cluster_resources()["CPU"] == 1 nodes = [] nodes += [cluster.add_node(resources=dict(CPU=1))] @@ -104,7 +105,7 @@ def test_counting_resources(start_connected_cluster): runner = TrialRunner(BasicVariantGenerator()) kwargs = {"stopping_criterion": {"training_iteration": 10}} - trials = [Trial("__fake", **kwargs), Trial("__fake", **kwargs)] + trials = [Trial("test2", **kwargs), Trial("test2", **kwargs)] for t in trials: runner.add_trial(t) @@ -304,7 +305,6 @@ def test_cluster_down_simple(start_connected_cluster, tmpdir): cluster = _start_new_cluster() runner = TrialRunner(BasicVariantGenerator()) runner.restore(dirpath) - print([t.status for t in runner.get_trials()]) runner.step() # start runner.step() # start2 From 07df20b6e814eba2157de5dc05038b85b0dcb95f Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 13:31:56 -0800 Subject: [PATCH 72/74] nit --- python/ray/tune/test/cluster_tests.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 66e732d18d52..b4dd98a052f2 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -43,7 +43,7 @@ def _save(self, path): def _restore(self, state): self.state = state - tune.register_trainable("test2", _Fail) + tune.register_trainable("test", _Fail) def _start_new_cluster(): @@ -105,7 +105,7 @@ def test_counting_resources(start_connected_cluster): runner = TrialRunner(BasicVariantGenerator()) kwargs = {"stopping_criterion": {"training_iteration": 10}} - trials = [Trial("test2", **kwargs), Trial("test2", **kwargs)] + trials = [Trial("test", **kwargs), Trial("test", **kwargs)] for t in trials: runner.add_trial(t) @@ -378,7 +378,7 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): {run_register_trainable_fn}() kwargs = dict( - run="test2", + run="test", stop=dict(training_iteration=5), checkpoint_freq=1, max_failures=1) From 1ff31cb70e137ca7fbed1f8648e23e46bfa9c525 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 17:26:06 -0800 Subject: [PATCH 73/74] Fix counting resources test --- python/ray/tune/test/cluster_tests.py | 20 +++++++------------- 1 file changed, 7 insertions(+), 13 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index b4dd98a052f2..66bdecebfed5 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -95,25 +95,24 @@ def start_connected_emptyhead_cluster(): def test_counting_resources(start_connected_cluster): """Tests that Tune accounting is consistent with actual cluster.""" cluster = start_connected_cluster - register_fail_trainable() # This is less flaky than _register_all - assert ray.global_state.cluster_resources()["CPU"] == 1 nodes = [] - nodes += [cluster.add_node(resources=dict(CPU=1))] - assert cluster.wait_for_nodes() - assert ray.global_state.cluster_resources()["CPU"] == 2 - + assert ray.global_state.cluster_resources()["CPU"] == 1 runner = TrialRunner(BasicVariantGenerator()) kwargs = {"stopping_criterion": {"training_iteration": 10}} - trials = [Trial("test", **kwargs), Trial("test", **kwargs)] + trials = [Trial("__fake", **kwargs), Trial("__fake", **kwargs)] for t in trials: runner.add_trial(t) runner.step() # run 1 + nodes += [cluster.add_node(resources=dict(CPU=1))] + assert cluster.wait_for_nodes() + assert ray.global_state.cluster_resources()["CPU"] == 2 cluster.remove_node(nodes.pop()) assert cluster.wait_for_nodes() assert ray.global_state.cluster_resources()["CPU"] == 1 runner.step() # run 2 + assert sum(t.status == Trial.RUNNING for t in runner.get_trials()) == 1 for i in range(5): nodes += [cluster.add_node(resources=dict(CPU=1))] @@ -121,12 +120,7 @@ def test_counting_resources(start_connected_cluster): assert ray.global_state.cluster_resources()["CPU"] == 6 runner.step() # 1 result - - for i in range(5): - node = nodes.pop() - cluster.remove_node(node) - assert cluster.wait_for_nodes() - assert ray.global_state.cluster_resources()["CPU"] == 1 + assert sum(t.status == Trial.RUNNING for t in runner.get_trials()) == 2 @pytest.mark.skip("Add this test once reconstruction is fixed") From 6998a0175e97c3092d4bc90b7041982357c0970a Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 17:47:19 -0800 Subject: [PATCH 74/74] better test and error msg --- python/ray/tune/test/cluster_tests.py | 7 +++---- python/ray/tune/test/trial_runner_test.py | 19 +++++++++++++++++++ python/ray/tune/tune.py | 4 +++- 3 files changed, 25 insertions(+), 5 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 66bdecebfed5..8019929d1593 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -316,11 +316,11 @@ def test_cluster_down_full(start_connected_cluster, tmpdir): """Tests that run_experiment restoring works on cluster shutdown.""" cluster = start_connected_cluster dirpath = str(tmpdir) + exp1_args = dict( run="__fake", stop=dict(training_iteration=3), - checkpoint_freq=1, - max_failures=1) + checkpoint_freq=1) exp2_args = dict(run="__fake", stop=dict(training_iteration=3)) exp3_args = dict( run="__fake", @@ -330,8 +330,7 @@ def test_cluster_down_full(start_connected_cluster, tmpdir): run="__fake", stop=dict(training_iteration=3), config=dict(mock_error=True), - checkpoint_freq=1, - max_failures=1) + checkpoint_freq=1) tune.run_experiments( dict(exp1=exp1_args, exp2=exp2_args, exp3=exp3_args, exp4=exp4_args), diff --git a/python/ray/tune/test/trial_runner_test.py b/python/ray/tune/test/trial_runner_test.py index 1015e3376d0a..d60ca7ee2b03 100644 --- a/python/ray/tune/test/trial_runner_test.py +++ b/python/ray/tune/test/trial_runner_test.py @@ -587,6 +587,25 @@ def train(config, reporter): self.assertEqual(trial.status, Trial.TERMINATED) self.assertEqual(trial.last_result[TIMESTEPS_TOTAL], 99) + def testSimultaneousExperimentRestore(self): + tmpdir = tempfile.mkdtemp() + def train(config, reporter): + for i in range(100): + reporter(timesteps_total=i) + + register_trainable("f1", train) + exp1 = Experiment(**{ + "name": "foo", + "run": "f1", + "config": { + "script_min_iter_time_s": 0 + } + }) + self.assertRaises( + AssertionError, lambda: run_experiments( + exp1, restore_from_path=tmpdir)) + shutil.rmtree(tmpdir) + def testExperimentList(self): def train(config, reporter): for i in range(100): diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index 328a4543ac07..86a4d2ba79d5 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -112,7 +112,9 @@ def run_experiments(experiments=None, queue_trials=queue_trials, trial_executor=trial_executor) - if restore_from_path and os.path.exists(restore_from_path): + if restore_from_path: + if not os.path.exists(restore_from_path): + raise ValueError("Provided path invalid: %s" % restore_from_path) assert experiments is None, ( "Simultaneous starting experiments and restoring not supported.") runner.restore(restore_from_path)