From e557b6a9a0e0878c47d3b103493b3b7960dac995 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Sun, 4 Nov 2018 21:44:53 -0800 Subject: [PATCH 001/172] [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 3c9ae43e6a78b..03430d3d0c93c 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 8e3eb861246f3..2ee8313671127 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 002/172] 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 c4cf2b801623a..fa55f9e1b1636 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 0000000000000..57268b28ee695 --- /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 380730fcb35fe..e47ee01a0893f 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 003/172] 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 57268b28ee695..065c8cac1d4b1 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 004/172] 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 c5501dc9c525a..33af028d836ca 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 005/172] 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 c4cf2b801623a..fbce577035c83 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 33af028d836ca..a4427283eed53 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 006/172] 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 688fbb3ae4809..73ed03112d737 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 42e65809e9972..b9ae8daa406ca 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 8eee516ab119f..066b4d2721a9f 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 7a7ff6aa7689b..692aa17c51f70 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 3e2483ee6b4e9..be67f685e8dac 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 f997566a5076e..e1971981d4db7 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 007/172] 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 b9ae8daa406ca..fc90c9909507f 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 692aa17c51f70..a5a31b08b9e49 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 008/172] 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 a4427283eed53..24d4bdc4a6cd9 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 009/172] 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 be67f685e8dac..93ccc28fdbe7c 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 e1971981d4db7..b12c3cfc2373d 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 010/172] 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 24d4bdc4a6cd9..2d64f627ec8cd 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 011/172] 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 73ed03112d737..c674f26cd3767 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 fc90c9909507f..8569b7d9510a2 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 012/172] docs --- python/ray/services.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/python/ray/services.py b/python/ray/services.py index 8569b7d9510a2..3888ec883cfdf 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 013/172] 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 c4cf2b801623a..befa87ee93db8 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 b0696f4b32851..bae4d79bff1a0 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 014/172] 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 201256c58e6ff..86ded3a36661a 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 015/172] 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 86ded3a36661a..1aa8ff6aa04fe 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 94a8d8a60f6fc..afa52abfd9113 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 307ae52e5531f..284bdb4361fd3 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 016/172] 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 1aa8ff6aa04fe..7091d4314bdda 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 017/172] 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 7091d4314bdda..a7b48163a06f7 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 018/172] 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 065c8cac1d4b1..0415834ef759b 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 019/172] 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 175449420e775..b646c58eb11b6 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 f59e426514a31..cf0869fd0bbce 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 1aa8ff6aa04fe..2c44a8bf0fb8e 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 020/172] 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 bae4d79bff1a0..f56ddd2e18b84 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 021/172] 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 0415834ef759b..d87402f26b3df 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 f56ddd2e18b84..98f851d0073c2 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 022/172] 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 8a5aab376b020..56940e33cbcfd 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 b646c58eb11b6..8f4e391fbea7f 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 023/172] 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 f56ddd2e18b84..339546be13596 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 024/172] 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 ee191c90cbf4b..afaf5dd21921c 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 2d64f627ec8cd..e2003d9dc06ff 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 025/172] 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 d87402f26b3df..a320b8313cc99 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 026/172] 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 d3662c157207f..293c1b8b6b04d 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 027/172] 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 a320b8313cc99..d96bc4725664e 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 59559ebbe2c29..38cc1dee1cac3 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 b961d12b8cbe4..bfbd8f697fc55 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 bb31345c8f5cb..dc08b07915c65 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 028/172] 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 d96bc4725664e..45b0c2fca8c48 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 029/172] 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 45b0c2fca8c48..15e76a3c33eaf 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 030/172] 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 15e76a3c33eaf..af9f0b119f97f 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 38cc1dee1cac3..c0de5c871b934 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 031/172] 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 c0de5c871b934..4ff43cd3f8394 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 bfbd8f697fc55..24f65f6f36e6b 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 032/172] 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 1282dce61e803..339546be13596 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 033/172] 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 4ff43cd3f8394..0ccd9074f3996 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 034/172] 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 af9f0b119f97f..ab9e1786dc5f6 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 0ccd9074f3996..56b3bc2308628 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 24f65f6f36e6b..bd36a84761e5a 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 035/172] 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 e2003d9dc06ff..8dc288736b5ff 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 036/172] 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 ab9e1786dc5f6..a17795e5cdf98 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 037/172] 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 4a216a60d2be9..9b152492ca801 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 ab9e1786dc5f6..07a62eccf1f56 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 56b3bc2308628..8e0526c7aab81 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 038/172] 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 07a62eccf1f56..4697ac69b4433 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 039/172] 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 dc08b07915c65..d94a136e96c9b 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 040/172] 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 d94a136e96c9b..e80b11ac912e4 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 041/172] 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 f73aa4a1ef8ce..87f7e026d8928 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 042/172] 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 c468cf01584be..6a3fa165b98f2 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 48f567d8542bc..c1de033359970 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 8e0526c7aab81..ab904f455a719 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 bd36a84761e5a..de8da35e54fa8 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 e80b11ac912e4..35b3888f07e23 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 043/172] 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 2f3c04e251af5..85f3f8c972c65 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 8e0526c7aab81..65683eeb53c71 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 044/172] 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 fc6aa0bc5929e..68805a8ec5612 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 4697ac69b4433..57aeed34fedb7 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 045/172] 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 80685b7d31544..9cdee4ff117eb 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 57aeed34fedb7..19bb101105bb3 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 046/172] 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 19bb101105bb3..e0f37844aa91e 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 047/172] 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 8de720882644c..83140ad7c4b75 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 3639322e1109c..f51cd29a2ef0b 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 3bdfcccc3aafc..b777d675cd9cc 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 335660ecb836a..b8a98d9424bf4 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 048/172] 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 83140ad7c4b75..1b3e256feb0d4 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 f51cd29a2ef0b..bd825ac1d5250 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 b2266dd294ad8..e0b541218bf19 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 b777d675cd9cc..ca1a851530251 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 b8a98d9424bf4..a13ce945a48b6 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 049/172] 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 e0f37844aa91e..f9425cc3e301a 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 050/172] 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 3866d5c5b96bb..325c762eded8e 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 051/172] 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 325c762eded8e..39dfaab3ab934 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 052/172] 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 bd825ac1d5250..337f4e2b02e7e 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 ca1a851530251..cde9cfbe2d80d 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 a13ce945a48b6..d77fd2b79c848 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 053/172] 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 7b725e05f342e..dbee770b44cd0 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 35c413e717bb4..8e6ef765cee36 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 6b142d354ec7f..6af30e87fd3a3 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 337f4e2b02e7e..1a9e4ff17a30c 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 e0b541218bf19..d67e7705a1b12 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 cde9cfbe2d80d..f1f0e9455d323 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 d77fd2b79c848..afefc03fc19f9 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 054/172] 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 39dfaab3ab934..5af6a5503aec4 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 6af30e87fd3a3..55582c8d8395a 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 f1f0e9455d323..f5ccc34db6370 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 055/172] 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 dbee770b44cd0..6f8595d5c92b9 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 55582c8d8395a..1992e437e294e 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 d67e7705a1b12..92410906153f2 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 f5ccc34db6370..ce0be05aa0393 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 056/172] 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 6f8595d5c92b9..11cbcd1d3fce8 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 1992e437e294e..2b9d6ff42fc4b 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 1a9e4ff17a30c..e555e95121012 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 ce0be05aa0393..bec7bee90ae0c 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 afefc03fc19f9..328a4543ac07d 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 057/172] 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 11cbcd1d3fce8..538376a5b5b36 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 2b9d6ff42fc4b..e47c2e83f2083 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 92410906153f2..43229d3e704fe 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 058/172] 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 e47c2e83f2083..5ee4ac5f6599b 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 059/172] 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 538376a5b5b36..19cff512144c8 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 5ee4ac5f6599b..08c3df80c5c9f 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 bec7bee90ae0c..b47e8bbc7a5e4 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 060/172] 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 5af6a5503aec4..f6d6711ec2554 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 061/172] 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 293c1b8b6b04d..3f1ea4922bc89 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 08c3df80c5c9f..c89037d07483a 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 062/172] 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 f6d6711ec2554..3d948394cdde2 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 063/172] 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 3d948394cdde2..34f48b3d18fad 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 064/172] 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 34f48b3d18fad..5e747d9ef9bac 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 c89037d07483a..1015e3376d0ae 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 065/172] 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 5e747d9ef9bac..558f88d4ec54c 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 43229d3e704fe..53b9d1d805c25 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 b47e8bbc7a5e4..548c68c51cc03 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 066/172] 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 558f88d4ec54c..0b01a863093c4 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 067/172] 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 0b01a863093c4..0c978d5b11d6c 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 068/172] 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 0c978d5b11d6c..488178995a76d 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 069/172] 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 488178995a76d..fc9d670a03e69 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 070/172] 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 fc9d670a03e69..97eaf6b7411d9 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 071/172] 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 97eaf6b7411d9..66e732d18d527 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 072/172] 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 66e732d18d527..b4dd98a052f22 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 073/172] 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 b4dd98a052f22..66bdecebfed5d 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 074/172] 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 66bdecebfed5d..8019929d15931 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 1015e3376d0ae..d60ca7ee2b035 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 328a4543ac07d..86a4d2ba79d5b 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) From fcbc6dea478074d4c00b185bc392d6dc55b501a2 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Sat, 24 Nov 2018 20:00:27 -0800 Subject: [PATCH 075/172] 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 | 70 +++++++++++++++--- python/ray/tune/tune.py | 3 +- 7 files changed, 162 insertions(+), 61 deletions(-) diff --git a/python/ray/tune/ray_trial_executor.py b/python/ray/tune/ray_trial_executor.py index 7b725e05f342e..dbee770b44cd0 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 35c413e717bb4..8e6ef765cee36 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 6b142d354ec7f..6af30e87fd3a3 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 65683eeb53c71..83da6e0eff390 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -281,10 +281,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 e0b541218bf19..d67e7705a1b12 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 98bbbcb71c647..53107cbe86ff4 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -11,8 +11,8 @@ 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.trial import Trial +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 @@ -116,6 +116,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._iteration += 1 else: for trial in self._trials: if trial.status == Trial.PENDING: @@ -297,24 +302,65 @@ 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 _try_recover(self, trial, error_msg): + 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.") + + 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 335660ecb836a..c7aa6e560edc1 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -8,6 +8,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, @@ -90,8 +91,6 @@ def run_experiments(experiments=None, if search_alg is None: search_alg = BasicVariantGenerator() - search_alg.add_configurations(experiments) - runner = TrialRunner( search_alg, scheduler=scheduler, From 5d8e414af9fd70edaa8faef89aafd4a7d043a516 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Sun, 25 Nov 2018 18:01:19 -0800 Subject: [PATCH 076/172] py2mock --- .travis/install-dependencies.sh | 4 ++-- python/ray/tune/test/trial_runner_test.py | 13 +++++++++---- 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/.travis/install-dependencies.sh b/.travis/install-dependencies.sh index 293c1b8b6b04d..3f1ea4922bc89 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 6af30e87fd3a3..2d40e8eeb90f1 100644 --- a/python/ray/tune/test/trial_runner_test.py +++ b/python/ray/tune/test/trial_runner_test.py @@ -3,9 +3,9 @@ from __future__ import print_function import os +import sys import time import unittest -from unittest import mock import ray from ray.rllib import _register_all @@ -26,6 +26,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): @@ -1133,15 +1138,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 9137de0d7cc58215a283c24fe9ad47e37afec33c Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 18:44:27 -0800 Subject: [PATCH 077/172] nit --- python/ray/tune/ray_trial_executor.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/python/ray/tune/ray_trial_executor.py b/python/ray/tune/ray_trial_executor.py index dbee770b44cd0..ec360852391dd 100644 --- a/python/ray/tune/ray_trial_executor.py +++ b/python/ray/tune/ray_trial_executor.py @@ -154,13 +154,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.""" From 445372493b58446cb2018ed23834b803213cc5be Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 17:26:06 -0800 Subject: [PATCH 078/172] Fix counting resources test --- python/ray/tune/test/cluster_tests.py | 19 +++++++------------ 1 file changed, 7 insertions(+), 12 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index f9425cc3e301a..d95c884a1d440 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -86,24 +86,24 @@ 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))] - 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))] @@ -111,12 +111,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 5a2449970880a640386c4fff8329642509754ae5 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 18:54:17 -0800 Subject: [PATCH 079/172] Remove extraneous changes --- python/ray/tune/trial.py | 10 ++++++---- python/ray/tune/trial_runner.py | 17 ++--------------- 2 files changed, 8 insertions(+), 19 deletions(-) diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index 83da6e0eff390..b6bdbd1d755b4 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.""" diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index 53107cbe86ff4..d77da2b85b8c3 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -116,11 +116,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: @@ -283,17 +278,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) 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) self.trial_executor.stop_trial(trial) else: assert False, "Invalid scheduling decision: {}".format( @@ -321,11 +313,6 @@ def _checkpoint_if_needed(self, trial): 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.") - def try_recover(self, trial, error_msg): """Tries to recover trial. From b750d4efdfada19704951756f96433d470d667b6 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 19:04:11 -0800 Subject: [PATCH 080/172] docs --- python/ray/tune/ray_trial_executor.py | 2 +- python/ray/tune/trial_runner.py | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/python/ray/tune/ray_trial_executor.py b/python/ray/tune/ray_trial_executor.py index ec360852391dd..3e460e80dc0a0 100644 --- a/python/ray/tune/ray_trial_executor.py +++ b/python/ray/tune/ray_trial_executor.py @@ -117,7 +117,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/trial_runner.py b/python/ray/tune/trial_runner.py index d77da2b85b8c3..a3aa25bf0024d 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -317,6 +317,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 14da6ec51f50f9205a70b79bd5c20d478e97be7c Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Sun, 25 Nov 2018 00:50:01 -0800 Subject: [PATCH 081/172] yapf --- python/ray/tune/trial.py | 4 ++-- python/ray/tune/trial_runner.py | 3 +-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index b6bdbd1d755b4..d3a4e81451032 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 a3aa25bf0024d..20a2bac2d2135 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -340,8 +340,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. From 394c0e941931eb8781b34631f967ef094342c77e Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 19:22:22 -0800 Subject: [PATCH 082/172] Lint and small changes to tests --- python/ray/tune/test/cluster_tests.py | 48 +++++++++++++---------- python/ray/tune/test/trial_runner_test.py | 4 +- 2 files changed, 31 insertions(+), 21 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index d95c884a1d440..348de35f7b098 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -10,23 +10,28 @@ except ImportError: pytest_timeout = None -from ray.test.cluster_utils import Cluster import ray from ray import tune +from ray.rllib import _register_all +from ray.test.cluster_utils import Cluster 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 register_fail_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"] >= 4: + assert False return {} def _save(self, path): @@ -35,13 +40,10 @@ def _save(self, path): def _restore(self, state): self.state = state - tune.register_trainable("test", _Train) + tune.register_trainable("test", _Fail) -@pytest.fixture -def start_connected_cluster(): - # Start the Ray processes. - +def _start_new_cluster(): cluster = Cluster( initialize_head=True, connect=True, @@ -51,7 +53,15 @@ def start_connected_cluster(): "num_heartbeats_timeout": 10 }) }) - register_test_trainable() + # Pytest doesn't play nicely with imports + _register_all() + return cluster + + +@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() @@ -71,17 +81,14 @@ def start_connected_emptyhead_cluster(): "num_heartbeats_timeout": 10 }) }) - register_test_trainable() + # 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") def test_counting_resources(start_connected_cluster): """Tests that Tune accounting is consistent with actual cluster.""" @@ -128,7 +135,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) @@ -174,7 +181,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 @@ -194,7 +201,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 @@ -211,7 +218,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 @@ -233,6 +240,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 = { @@ -243,7 +251,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) diff --git a/python/ray/tune/test/trial_runner_test.py b/python/ray/tune/test/trial_runner_test.py index 2d40e8eeb90f1..8e4aa2cea1481 100644 --- a/python/ray/tune/test/trial_runner_test.py +++ b/python/ray/tune/test/trial_runner_test.py @@ -852,14 +852,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] @@ -868,6 +869,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() From 48fd3c3fcbf0e6fbd1fd559852095d12ab157a54 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 19:23:45 -0800 Subject: [PATCH 083/172] lint --- python/ray/tune/trial_runner.py | 2 +- python/ray/tune/tune.py | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index 20a2bac2d2135..d36bc1dec907c 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -11,7 +11,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 diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index c7aa6e560edc1..7840fa4fbdc23 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -8,7 +8,6 @@ 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, From bcf40513722b34a60206f00513a1bb074fbf6896 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 19:27:23 -0800 Subject: [PATCH 084/172] nit --- python/ray/tune/tune.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index 7840fa4fbdc23..335660ecb836a 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -90,6 +90,8 @@ def run_experiments(experiments=None, if search_alg is None: search_alg = BasicVariantGenerator() + search_alg.add_configurations(experiments) + runner = TrialRunner( search_alg, scheduler=scheduler, From 0f67265a9fd6c1c39525a5c092f5eb29f946db1d Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 20:52:16 -0800 Subject: [PATCH 085/172] small extraneous removals --- python/ray/tune/test/cluster_tests.py | 25 ------------------------- python/ray/tune/trial_runner.py | 7 ++----- 2 files changed, 2 insertions(+), 30 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 348de35f7b098..59f12181b8ff9 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -3,7 +3,6 @@ from __future__ import print_function import json -import time import pytest try: import pytest_timeout @@ -11,7 +10,6 @@ pytest_timeout = None import ray -from ray import tune from ray.rllib import _register_all from ray.test.cluster_utils import Cluster from ray.tune.error import TuneError @@ -20,29 +18,6 @@ from ray.tune.suggest import BasicVariantGenerator -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"] >= 4: - assert False - return {} - - def _save(self, path): - return self.state - - def _restore(self, state): - self.state = state - - tune.register_trainable("test", _Fail) - - def _start_new_cluster(): cluster = Cluster( initialize_head=True, diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index d36bc1dec907c..b3ac14b98bc1d 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -297,18 +297,15 @@ def _process_events(self): 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, error=True, error_msg=error_msg) 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) From 74b6a939d47432d36306642218f3e14142860822 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 26 Nov 2018 20:58:55 -0800 Subject: [PATCH 086/172] fix some merge? --- python/ray/tune/trial_runner.py | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index 548c68c51cc03..f64092cf2cf35 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -360,13 +360,11 @@ def _process_events(self): 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, error=True, error_msg=error_msg) def _checkpoint_if_needed(self, trial): """Checkpoints trial based off trial.last_result.""" From 3d0a2e32c63a6b63d9e46066e378072b29931770 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Fri, 30 Nov 2018 12:27:38 -0800 Subject: [PATCH 087/172] try recover --- python/ray/tune/ray_trial_executor.py | 2 +- python/ray/tune/trial_runner.py | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/python/ray/tune/ray_trial_executor.py b/python/ray/tune/ray_trial_executor.py index 19cff512144c8..451f5638da6cb 100644 --- a/python/ray/tune/ray_trial_executor.py +++ b/python/ray/tune/ray_trial_executor.py @@ -135,7 +135,7 @@ def start_trial(self, trial, checkpoint=None, raise_on_failure=False): self._stop_trial(trial, error=True, error_msg=error_msg) try: self._start_trial(trial, checkpoint) - except Exception as exc: + except Exception: logger.exception("Error starting runner, aborting!") error_msg = traceback.format_exc() self._stop_trial(trial, error=True, error_msg=error_msg) diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index 9cf597e3b4b85..3adcde22da793 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -359,7 +359,7 @@ def _process_events(self): error_msg = traceback.format_exc() if trial.status == Trial.RUNNING: if trial.should_recover(): - self.try_recover(trial, error_msg) + self._try_recover(trial, error_msg) else: self._scheduler_alg.on_trial_error(self, trial) self._search_alg.on_trial_complete( @@ -375,7 +375,7 @@ def _checkpoint_if_needed(self, trial): self.trial_executor.save(trial, storage=Checkpoint.DISK) self.trial_executor.try_checkpoint_metadata(trial) - 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. From 4bb938f4522e00b379aaaba1ac3f1c0ef9e2d45e Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 4 Dec 2018 15:18:13 -0800 Subject: [PATCH 088/172] merge --- .travis/install-dependencies.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.travis/install-dependencies.sh b/.travis/install-dependencies.sh index d7b68ecc6daea..5bae4ba87f8db 100755 --- a/.travis/install-dependencies.sh +++ b/.travis/install-dependencies.sh @@ -24,7 +24,7 @@ if [[ "$PYTHON" == "2.7" ]] && [[ "$platform" == "linux" ]]; then wget https://repo.continuum.io/miniconda/Miniconda2-4.5.4-Linux-x86_64.sh -O miniconda.sh -nv 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 \ + pip install -q cython==0.27.3 cmake tensorflow gym opencv-python pyyaml pandas==0.23.4 requests \ feather-format lxml openpyxl xlrd py-spy setproctitle faulthandler pytest-timeout mock elif [[ "$PYTHON" == "3.5" ]] && [[ "$platform" == "linux" ]]; then sudo apt-get update @@ -50,7 +50,7 @@ elif [[ "$PYTHON" == "2.7" ]] && [[ "$platform" == "macosx" ]]; then wget https://repo.continuum.io/miniconda/Miniconda2-4.5.4-MacOSX-x86_64.sh -O miniconda.sh -nv 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 \ + pip install -q cython==0.27.3 cmake tensorflow gym opencv-python pyyaml pandas==0.23.4 requests \ feather-format lxml openpyxl xlrd py-spy setproctitle faulthandler pytest-timeout mock elif [[ "$PYTHON" == "3.5" ]] && [[ "$platform" == "macosx" ]]; then # check that brew is installed From ac5d8c04e9cc3fa41fd6f314723df69b97d134a2 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Fri, 30 Nov 2018 12:17:30 -0800 Subject: [PATCH 089/172] Removed error raising --- python/ray/tune/ray_trial_executor.py | 8 +------- python/ray/tune/test/ray_trial_executor_test.py | 3 --- python/ray/tune/trial_executor.py | 3 +-- python/ray/tune/trial_runner.py | 5 +++-- 4 files changed, 5 insertions(+), 14 deletions(-) diff --git a/python/ray/tune/ray_trial_executor.py b/python/ray/tune/ray_trial_executor.py index 451f5638da6cb..3c3a5c509d6bc 100644 --- a/python/ray/tune/ray_trial_executor.py +++ b/python/ray/tune/ray_trial_executor.py @@ -110,7 +110,7 @@ def _stop_trial(self, trial, error=False, error_msg=None, if stop_logger: trial.close_logger() - def start_trial(self, trial, checkpoint=None, raise_on_failure=False): + def start_trial(self, trial, checkpoint=None): """Starts the trial. Will not return resources if trial repeatedly fails on start. @@ -119,10 +119,6 @@ def start_trial(self, trial, checkpoint=None, raise_on_failure=False): 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) @@ -141,8 +137,6 @@ def start_trial(self, trial, checkpoint=None, raise_on_failure=False): 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] diff --git a/python/ray/tune/test/ray_trial_executor_test.py b/python/ray/tune/test/ray_trial_executor_test.py index 8e6ef765cee36..86c4bb189595f 100644 --- a/python/ray/tune/test/ray_trial_executor_test.py +++ b/python/ray/tune/test/ray_trial_executor_test.py @@ -56,9 +56,6 @@ def testStartFailure(self): 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.""" diff --git a/python/ray/tune/trial_executor.py b/python/ray/tune/trial_executor.py index 53b9d1d805c25..7c0de0caed30f 100644 --- a/python/ray/tune/trial_executor.py +++ b/python/ray/tune/trial_executor.py @@ -70,14 +70,13 @@ def has_resources(self, resources): raise NotImplementedError("Subclasses of TrialExecutor must provide " "has_resources() method") - def start_trial(self, trial, checkpoint=None, raise_on_failure=False): + def start_trial(self, trial, checkpoint=None): """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") diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index 3adcde22da793..80a89901ce9f9 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -394,12 +394,13 @@ def _try_recover(self, trial, error_msg): 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) + self.trial_executor.start_trial(trial) + if trial.status == Trial.ERROR: + raise RuntimeError("Trial did not start correctly.") else: logger.debug("Notifying Scheduler and requeueing trial.") self._requeue_trial(trial) except Exception: - 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) From 83e1a26bcc2def795786a20b3bb4708221d32803 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 4 Dec 2018 15:55:25 -0800 Subject: [PATCH 090/172] Rename checkpoint_mode --- python/ray/tune/ray_trial_executor.py | 4 ++-- python/ray/tune/trial_executor.py | 8 +++++--- python/ray/tune/trial_runner.py | 8 ++++---- python/ray/tune/tune.py | 2 -- 4 files changed, 11 insertions(+), 11 deletions(-) diff --git a/python/ray/tune/ray_trial_executor.py b/python/ray/tune/ray_trial_executor.py index 3c3a5c509d6bc..a214795f53e6c 100644 --- a/python/ray/tune/ray_trial_executor.py +++ b/python/ray/tune/ray_trial_executor.py @@ -19,8 +19,8 @@ class RayTrialExecutor(TrialExecutor): """An implemention of TrialExecutor based on Ray.""" - def __init__(self, queue_trials=False, track_checkpoints=False): - super(RayTrialExecutor, self).__init__(queue_trials, track_checkpoints) + def __init__(self, queue_trials=False, checkpoint_mode=False): + super(RayTrialExecutor, self).__init__(queue_trials, checkpoint_mode) self._running = {} # Since trial resume after paused should not run # trial.train.remote(), thus no more new remote object id generated. diff --git a/python/ray/tune/trial_executor.py b/python/ray/tune/trial_executor.py index 7c0de0caed30f..5ff39e78b62cd 100644 --- a/python/ray/tune/trial_executor.py +++ b/python/ray/tune/trial_executor.py @@ -16,7 +16,7 @@ class TrialExecutor(object): and starting/stopping trials. """ - def __init__(self, queue_trials=False, track_checkpoints=False): + def __init__(self, queue_trials=False, checkpoint_mode=False): """Initializes a new TrialExecutor. Args: @@ -24,9 +24,11 @@ def __init__(self, queue_trials=False, track_checkpoints=False): not currently have enough resources to launch one. This should be set to True when running on an autoscaling cluster to enable automatic scale-up. + checkpoint_mode (bool): Whether to track metadata on status + change. """ self._queue_trials = queue_trials - self._track_checkpoints = track_checkpoints + self._checkpoint_mode = checkpoint_mode self._checkpoints = {} def set_status(self, trial, status): @@ -50,7 +52,7 @@ def try_checkpoint_metadata(self, trial): Args: trial (Trial): Trial to checkpoint. """ - if self._track_checkpoints and trial.checkpoint_freq > 0: + if self._checkpoint_mode and trial.checkpoint_freq > 0: if trial._checkpoint.storage == Checkpoint.MEMORY: logger.debug("Not saving data for trial w/ memory checkpoint.") return diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index 80a89901ce9f9..39c1b7f9422ef 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -84,7 +84,7 @@ def __init__(self, self._trials = [] self.trial_executor = trial_executor or \ RayTrialExecutor(queue_trials=queue_trials, - track_checkpoints=checkpoint_freq > 0) + checkpoint_mode=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. @@ -109,13 +109,12 @@ def save(self): 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 = { "checkpoints": list( self.trial_executor.get_checkpoints().values()), "total_time": self._total_time, - "stop_queue": self._stop_queue + "stop_queue": self._stop_queue, + "iteration": self._iteration } with open(os.path.join(checkpoint_dir, "experiment.state"), "wb") as f: pickle.dump(runner_state, f) @@ -145,6 +144,7 @@ def restore(self, checkpoint_dir): self._total_time = runner_state["total_time"] self._stop_queue = runner_state["stop_queue"] + self._iteration = runner_state["iteration"] def is_finished(self): """Returns whether all trials have finished running.""" diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index fa773572c1102..3c1b43b5bc195 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -100,8 +100,6 @@ def run_experiments(experiments=None, if search_alg is None: search_alg = BasicVariantGenerator() - search_alg.add_configurations(experiments) - runner = TrialRunner( search_alg, scheduler=scheduler, From d77b93489f4372bfaebab22520da716a3cba8709 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 4 Dec 2018 15:59:04 -0800 Subject: [PATCH 091/172] note for pickling --- python/ray/tune/trial.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index 74449aa3feda9..4c0cc41adff68 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -329,8 +329,13 @@ def __str__(self): return identifier def __getstate__(self): + """Memento generator for Trial. + + Sets RUNNING trials to PENDING, and flushes the result logger. + Note this can only occur if the trial holds a DISK checkpoint. + """ if not self._checkpoint.storage == Checkpoint.DISK: - raise ValueError("Most recent checkpoint cannot be in-memory.") + raise ValueError("Checkpoint cannot be in-memory.") state = self.__dict__.copy() if state["status"] == Trial.RUNNING: From f3071eb72e84e8b0f1b5d30d9c14744712e1b84c Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 4 Dec 2018 18:13:36 -0800 Subject: [PATCH 092/172] Better UI --- python/ray/tune/test/cluster_tests.py | 2 +- python/ray/tune/trial_runner.py | 80 ++++++++++++++++++++------- python/ray/tune/tune.py | 36 ++++++------ 3 files changed, 79 insertions(+), 39 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 8019929d15931..e4edd7931b415 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -291,7 +291,7 @@ def test_cluster_down_simple(start_connected_cluster, tmpdir): runner.step() # start2 runner.step() # step assert all(t.status == Trial.RUNNING for t in runner.get_trials()) - runner.save() + runner.checkpoint() cluster.shutdown() ray.shutdown() diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index 39c1b7f9422ef..6c252ee78dfe9 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -81,7 +81,6 @@ def __init__(self, """ self._search_alg = search_alg self._scheduler_alg = scheduler or FIFOScheduler() - self._trials = [] self.trial_executor = trial_executor or \ RayTrialExecutor(queue_trials=queue_trials, checkpoint_mode=checkpoint_freq > 0) @@ -92,17 +91,21 @@ def __init__(self, os.environ.get("TRIALRUNNER_WALLTIME_LIMIT", float('inf'))) self._total_time = 0 self._iteration = 0 + self._verbose = verbose + self._queue_trials = queue_trials + self._server = None if launch_web_server: + self._server_port = server_port self._server = TuneServer(self, server_port) + + self._trials = [] self._stop_queue = [] - self._verbose = verbose - self._queue_trials = queue_trials + self._trial_checkpoints = {} self._checkpoint_dir = checkpoint_dir self._checkpoint_freq = checkpoint_freq - self._trial_checkpoints = {} - def save(self): + def checkpoint(self): """Saves all trial checkpoints to `self._checkpoint_dir.`""" checkpoint_dir = self._checkpoint_dir if not os.path.exists(checkpoint_dir): @@ -112,16 +115,15 @@ def save(self): runner_state = { "checkpoints": list( self.trial_executor.get_checkpoints().values()), - "total_time": self._total_time, - "stop_queue": self._stop_queue, - "iteration": self._iteration + "runner": self } 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): + @classmethod + def restore(cls, checkpoint_dir, trial_executor=None): """Restores all checkpointed trials from previous run. Requires user to manually re-register their objects. Also stops @@ -129,22 +131,33 @@ def restore(self, checkpoint_dir): Args: checkpoint_dir (str): Path to checkpoint (previously specified). - """ - logger.debug("Stopping all trials.") - for trial in self._trials: - self.stop_trial(trial) + Returns: + runner (TrialRunner): A TrialRunner to resume experiments from. + """ with open(os.path.join(checkpoint_dir, "experiment.state"), "rb") as f: runner_state = pickle.load(f) - logger.info("Replacing all trials with checkpoint state.") + runner = runner_state["runner"] + + logger.warning( + "Tune recovery is still experimental." + "There is limited search algorithm recovery support." + "Restoring with a BasicVariantGenerator and FIFOScheduler.") + + from ray.tune.suggest import BasicVariantGenerator + runner._search_alg = BasicVariantGenerator() + runner._scheduler_alg = FIFOScheduler() + + runner.trial_executor = trial_executor or \ + RayTrialExecutor(queue_trials=runner._queue_trials, + checkpoint_mode=runner._checkpoint_freq > 0) + + logger.info("Adding all trials with checkpoint state.") for ckpt in runner_state["checkpoints"]: trial = pickle.loads(ckpt) - self.add_trial(trial) - - self._total_time = runner_state["total_time"] - self._stop_queue = runner_state["stop_queue"] - self._iteration = runner_state["iteration"] + runner.add_trial(trial) + return runner def is_finished(self): """Returns whether all trials have finished running.""" @@ -194,7 +207,7 @@ def step(self): if self._checkpoint_freq: if (self._iteration % self._checkpoint_freq == 0 or self.is_finished()): - self.save() + self.checkpoint() self._iteration += 1 @@ -481,3 +494,30 @@ def stop_trial(self, trial): error = True self.trial_executor.stop_trial(trial, error=error, error_msg=error_msg) + + + def __getstate__(self): + state = self.__dict__.copy() + + state["trial_executor"] = None + + if not isinstance(state["_scheduler_alg"], FIFOScheduler): + # TODO(rliaw): Remove this once component FT is implemented + state["_scheduler_alg"] = + + state["_search_alg"] = None + + if state["_server"]: + state["_launch_web_server"] = True + state["_stop_queue"] = [] + state["_trials"] = [] + state["_trial_checkpoints"] = {} + return state + + + def __setstate__(self, state): + if "_launch_web_server" in state: + state.pop("_launch_web_server") + state["_server"] = TuneServer(self, state["_server_port"]) + + self.__dict__.update(state) diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index 3c1b43b5bc195..822e0809f762f 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -94,32 +94,32 @@ def run_experiments(experiments=None, """ - if scheduler is None: - scheduler = FIFOScheduler() - - if search_alg is None: - search_alg = BasicVariantGenerator() - - runner = TrialRunner( - search_alg, - scheduler=scheduler, - checkpoint_dir=checkpoint_dir or DEFAULT_RESULTS_DIR, - checkpoint_freq=checkpoint_freq, - launch_web_server=with_server, - server_port=server_port, - verbose=verbose, - queue_trials=queue_trials, - trial_executor=trial_executor) - 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) + runner = TrialRunner.restore(restore_from_path) else: + if scheduler is None: + scheduler = FIFOScheduler() + + if search_alg is None: + search_alg = BasicVariantGenerator() + search_alg.add_configurations(experiments) + runner = TrialRunner( + search_alg, + scheduler=scheduler, + checkpoint_dir=checkpoint_dir or DEFAULT_RESULTS_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)) last_debug = 0 while not runner.is_finished(): From 0263e93bd3ad8e0451232a05814647db734b54a8 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 4 Dec 2018 18:15:12 -0800 Subject: [PATCH 093/172] Fix lint --- python/ray/tune/test/cluster_tests.py | 4 +--- python/ray/tune/test/trial_runner_test.py | 5 +++-- python/ray/tune/trial_runner.py | 4 +--- python/ray/tune/tune.py | 1 + 4 files changed, 6 insertions(+), 8 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index e4edd7931b415..9151370f318cf 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -318,9 +318,7 @@ def test_cluster_down_full(start_connected_cluster, tmpdir): dirpath = str(tmpdir) exp1_args = dict( - run="__fake", - stop=dict(training_iteration=3), - checkpoint_freq=1) + run="__fake", stop=dict(training_iteration=3), checkpoint_freq=1) exp2_args = dict(run="__fake", stop=dict(training_iteration=3)) exp3_args = dict( run="__fake", diff --git a/python/ray/tune/test/trial_runner_test.py b/python/ray/tune/test/trial_runner_test.py index d60ca7ee2b035..099e03e2b89af 100644 --- a/python/ray/tune/test/trial_runner_test.py +++ b/python/ray/tune/test/trial_runner_test.py @@ -589,6 +589,7 @@ def train(config, reporter): def testSimultaneousExperimentRestore(self): tmpdir = tempfile.mkdtemp() + def train(config, reporter): for i in range(100): reporter(timesteps_total=i) @@ -602,8 +603,8 @@ def train(config, reporter): } }) self.assertRaises( - AssertionError, lambda: run_experiments( - exp1, restore_from_path=tmpdir)) + AssertionError, + lambda: run_experiments(exp1, restore_from_path=tmpdir)) shutil.rmtree(tmpdir) def testExperimentList(self): diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index 6c252ee78dfe9..672b8d7645aee 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -495,7 +495,6 @@ def stop_trial(self, trial): self.trial_executor.stop_trial(trial, error=error, error_msg=error_msg) - def __getstate__(self): state = self.__dict__.copy() @@ -503,7 +502,7 @@ def __getstate__(self): if not isinstance(state["_scheduler_alg"], FIFOScheduler): # TODO(rliaw): Remove this once component FT is implemented - state["_scheduler_alg"] = + state["_scheduler_alg"] = None state["_search_alg"] = None @@ -514,7 +513,6 @@ def __getstate__(self): state["_trial_checkpoints"] = {} return state - def __setstate__(self, state): if "_launch_web_server" in state: state.pop("_launch_web_server") diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index 822e0809f762f..fcb006d9c166e 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -14,6 +14,7 @@ from ray.tune.schedulers import (HyperBandScheduler, AsyncHyperBandScheduler, FIFOScheduler, MedianStoppingRule) from ray.tune.web_server import TuneServer +from ray.tune.result import DEFAULT_RESULTS_DIR logger = logging.getLogger(__name__) From 71db5df5b8a96b12b6fea81cd066846a5e8b9d94 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 4 Dec 2018 18:16:40 -0800 Subject: [PATCH 094/172] nit --- python/ray/tune/tune.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index fcb006d9c166e..0fcd9c71a0499 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -98,9 +98,9 @@ def run_experiments(experiments=None, 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, ( + assert not experiments, ( "Simultaneous starting experiments and restoring not supported.") - runner = TrialRunner.restore(restore_from_path) + runner = TrialRunner.restore(restore_from_path, trial_executor) else: if scheduler is None: scheduler = FIFOScheduler() From 9b9e771f119efb1a78349422325bf7bd9f699dec Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Thu, 6 Dec 2018 16:38:45 -0800 Subject: [PATCH 095/172] fix up tests --- python/ray/tune/test/cluster_tests.py | 12 ++++-------- python/ray/tune/test/trial_runner_test.py | 8 +++----- python/ray/tune/trial_runner.py | 6 +++--- 3 files changed, 10 insertions(+), 16 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 9151370f318cf..1358b595d8d11 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -297,8 +297,7 @@ def test_cluster_down_simple(start_connected_cluster, tmpdir): ray.shutdown() cluster = _start_new_cluster() - runner = TrialRunner(BasicVariantGenerator()) - runner.restore(dirpath) + runner = TrialRunner.restore(dirpath) runner.step() # start runner.step() # start2 @@ -341,8 +340,7 @@ def test_cluster_down_full(start_connected_cluster, tmpdir): cluster = _start_new_cluster() # Check that last_result.iteration = 1 - runner = TrialRunner(BasicVariantGenerator()) - runner.restore(dirpath) + runner = TrialRunner.restore(dirpath) trials = runner.get_trials() trials = tune.run_experiments(restore_from_path=dirpath) assert len(trials) == 2 @@ -393,8 +391,7 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): 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) + runner = TrialRunner.restore(dirpath) trials = runner.get_trials() last_res = trials[0].last_result if last_res is not None and last_res["training_iteration"] == 3: @@ -407,8 +404,7 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): register_fail_trainable() # Inspect the internal trialrunner just in case - runner = TrialRunner(BasicVariantGenerator()) - runner.restore(dirpath) + runner = TrialRunner.restore(dirpath) trials = runner.get_trials() assert trials[0].last_result["training_iteration"] == 3 assert trials[0].status == Trial.PENDING diff --git a/python/ray/tune/test/trial_runner_test.py b/python/ray/tune/test/trial_runner_test.py index 099e03e2b89af..b6bd8af91c797 100644 --- a/python/ray/tune/test/trial_runner_test.py +++ b/python/ray/tune/test/trial_runner_test.py @@ -1577,7 +1577,7 @@ def _suggest(self, trial_id): self.assertRaises(TuneError, runner.step) def testSaveRestore(self): - """Creates trials of different status to test runner.save/restore.""" + """Creates trials of different status to test runner.checkpoint/restore.""" ray.init(num_cpus=3) tmpdir = tempfile.mkdtemp() default_resources = Resources(cpu=1, gpu=0) @@ -1625,8 +1625,7 @@ def testSaveRestore(self): self.assertEquals(len(runner.trial_executor.get_checkpoints()), 3) self.assertEquals(trials[2].status, Trial.RUNNING) - runner2 = TrialRunner(BasicVariantGenerator()) - runner2.restore(tmpdir) + runner2 = TrialRunner.restore(tmpdir) for tid in ["trial_terminate", "trial_fail"]: original_trial = runner.get_trial(tid) restored_trial = runner2.get_trial(tid) @@ -1660,8 +1659,7 @@ def testNoSave(self): runner.step() # start runner.step() - runner2 = TrialRunner(BasicVariantGenerator()) - runner2.restore(tmpdir) + runner2 = TrialRunner.restore(tmpdir) self.assertEquals(len(runner2.get_trials()), 0) runner2.step() self.assertRaises(TuneError, runner2.step) diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index 672b8d7645aee..b6a763845886c 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -499,15 +499,15 @@ def __getstate__(self): state = self.__dict__.copy() state["trial_executor"] = None - + state["_search_alg"] = None if not isinstance(state["_scheduler_alg"], FIFOScheduler): # TODO(rliaw): Remove this once component FT is implemented state["_scheduler_alg"] = None - state["_search_alg"] = None - if state["_server"]: state["_launch_web_server"] = True + state["_server"] = None + state["_stop_queue"] = [] state["_trials"] = [] state["_trial_checkpoints"] = {} From d669463c9fc3e6f22431b0c3d069b7014cb37292 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Thu, 6 Dec 2018 16:40:48 -0800 Subject: [PATCH 096/172] note --- python/ray/tune/trial_runner.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index b6a763845886c..a8f8bac071155 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -129,6 +129,8 @@ def restore(cls, checkpoint_dir, trial_executor=None): Requires user to manually re-register their objects. Also stops all ongoing trials. + TODO: Consider checkpointing registry too. + Args: checkpoint_dir (str): Path to checkpoint (previously specified). From 55562a5b6d550f4f98b96fa780e6bcde13bea3c6 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Thu, 6 Dec 2018 16:41:49 -0800 Subject: [PATCH 097/172] nit --- python/ray/tune/trial_runner.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index a8f8bac071155..8e881cee41129 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -140,13 +140,12 @@ def restore(cls, checkpoint_dir, trial_executor=None): with open(os.path.join(checkpoint_dir, "experiment.state"), "rb") as f: runner_state = pickle.load(f) - runner = runner_state["runner"] - logger.warning( "Tune recovery is still experimental." "There is limited search algorithm recovery support." "Restoring with a BasicVariantGenerator and FIFOScheduler.") + runner = runner_state["runner"] from ray.tune.suggest import BasicVariantGenerator runner._search_alg = BasicVariantGenerator() runner._scheduler_alg = FIFOScheduler() From 6dd5e59734aa1e59379beefa26f026c6b9100ba3 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Fri, 7 Dec 2018 00:46:23 -0800 Subject: [PATCH 098/172] text --- python/ray/tune/trial_runner.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index 8e881cee41129..820bd2ad8a021 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -141,8 +141,8 @@ def restore(cls, checkpoint_dir, trial_executor=None): runner_state = pickle.load(f) logger.warning( - "Tune recovery is still experimental." - "There is limited search algorithm recovery support." + "Tune recovery is still experimental. " + "There is limited search algorithm recovery support. " "Restoring with a BasicVariantGenerator and FIFOScheduler.") runner = runner_state["runner"] From 0c3ade932b006977e3f26bbc43e77e5f0c1f3fdf Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Fri, 7 Dec 2018 12:46:42 -0800 Subject: [PATCH 099/172] nit --- python/ray/tune/test/trial_runner_test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/ray/tune/test/trial_runner_test.py b/python/ray/tune/test/trial_runner_test.py index b6bd8af91c797..3ac2f8830e563 100644 --- a/python/ray/tune/test/trial_runner_test.py +++ b/python/ray/tune/test/trial_runner_test.py @@ -1576,7 +1576,7 @@ def _suggest(self, trial_id): self.assertTrue(searcher.is_finished()) self.assertRaises(TuneError, runner.step) - def testSaveRestore(self): + def testTrialSaveRestore(self): """Creates trials of different status to test runner.checkpoint/restore.""" ray.init(num_cpus=3) tmpdir = tempfile.mkdtemp() @@ -1640,7 +1640,7 @@ def testSaveRestore(self): self.assertRaises(TuneError, runner2.step) shutil.rmtree(tmpdir) - def testNoSave(self): + def testTrialNoSave(self): """Check that non-checkpointing trials are not saved.""" ray.init(num_cpus=3) tmpdir = tempfile.mkdtemp() From e1e7b4e4bc003001ee47187058793602112cf652 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Sat, 8 Dec 2018 15:04:33 -0800 Subject: [PATCH 100/172] fix --- python/ray/tune/test/cluster_tests.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 1358b595d8d11..dd111dd164d65 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -273,6 +273,8 @@ 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)) + assert cluster.wait_for_nodes() + dirpath = str(tmpdir) runner = TrialRunner( BasicVariantGenerator(), checkpoint_freq=2, checkpoint_dir=dirpath) From 8961de115f453a311aac48b3f6e159d6151fb3cd Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 11 Dec 2018 01:22:55 -0800 Subject: [PATCH 101/172] fix usability --- python/ray/tune/test/cluster_tests.py | 6 +++--- python/ray/tune/test/trial_runner_test.py | 22 +--------------------- python/ray/tune/trial_runner.py | 17 ++++++++++++++--- python/ray/tune/tune.py | 22 +++++++++------------- 4 files changed, 27 insertions(+), 40 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index dd111dd164d65..3498b0ce434c4 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -344,7 +344,7 @@ def test_cluster_down_full(start_connected_cluster, tmpdir): # Check that last_result.iteration = 1 runner = TrialRunner.restore(dirpath) trials = runner.get_trials() - trials = tune.run_experiments(restore_from_path=dirpath) + trials = tune.run_experiments(checkpoint_dir=dirpath) assert len(trials) == 2 assert all(t.status in [Trial.TERMINATED, Trial.ERROR] for t in trials) cluster.shutdown() @@ -391,7 +391,7 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): # 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")): + if os.path.exists(os.path.join(dirpath, TrialRunner.CKPT_FILE)): # Inspect the internal trialrunner runner = TrialRunner.restore(dirpath) trials = runner.get_trials() @@ -413,6 +413,6 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): # Restore properly from checkpoint trials = tune.run_experiments( - restore_from_path=dirpath, raise_on_failed_trial=False) + checkpoint_dir=dirpath, raise_on_failed_trial=False) assert all(t.status == Trial.ERROR for t in trials) cluster.shutdown() diff --git a/python/ray/tune/test/trial_runner_test.py b/python/ray/tune/test/trial_runner_test.py index 3ac2f8830e563..302b00864de9a 100644 --- a/python/ray/tune/test/trial_runner_test.py +++ b/python/ray/tune/test/trial_runner_test.py @@ -587,26 +587,6 @@ 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): @@ -1577,7 +1557,7 @@ def _suggest(self, trial_id): self.assertRaises(TuneError, runner.step) def testTrialSaveRestore(self): - """Creates trials of different status to test runner.checkpoint/restore.""" + """Creates different trials to test runner.checkpoint/restore.""" ray.init(num_cpus=3) tmpdir = tempfile.mkdtemp() default_resources = Resources(cpu=1, gpu=0) diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index 820bd2ad8a021..41afe44cfbd16 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -50,6 +50,8 @@ class TrialRunner(object): misleading benchmark results. """ + CKPT_FILE = "experiment.state" + def __init__(self, search_alg, scheduler=None, @@ -67,7 +69,8 @@ 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_dir (str): Path where global checkpoints are stored + and restored from. checkpoint_freq (int): How many steps between global checkpoints. A value of 0 (default) disables checkpointing. server_port (int): Port number for launching TuneServer @@ -104,6 +107,12 @@ def __init__(self, self._trial_checkpoints = {} self._checkpoint_dir = checkpoint_dir self._checkpoint_freq = checkpoint_freq + if self._checkpoint_freq and not self._checkpoint_dir: + logger.warning("No checkpoint directory specified - turning " + "off checkpointing.") + self._checkpoint_freq = 0 + + assert self._checkpoint_freq >= 0, "checkpoint_freq must be positive." def checkpoint(self): """Saves all trial checkpoints to `self._checkpoint_dir.`""" @@ -117,7 +126,8 @@ def checkpoint(self): self.trial_executor.get_checkpoints().values()), "runner": self } - with open(os.path.join(checkpoint_dir, "experiment.state"), "wb") as f: + with open(os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE), + "wb") as f: pickle.dump(runner_state, f) return checkpoint_dir @@ -137,7 +147,8 @@ def restore(cls, checkpoint_dir, trial_executor=None): Returns: runner (TrialRunner): A TrialRunner to resume experiments from. """ - with open(os.path.join(checkpoint_dir, "experiment.state"), "rb") as f: + with open(os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE), + "rb") as f: runner_state = pickle.load(f) logger.warning( diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index 0fcd9c71a0499..9f62ed9d0c755 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -14,7 +14,6 @@ from ray.tune.schedulers import (HyperBandScheduler, AsyncHyperBandScheduler, FIFOScheduler, MedianStoppingRule) from ray.tune.web_server import TuneServer -from ray.tune.result import DEFAULT_RESULTS_DIR logger = logging.getLogger(__name__) @@ -37,7 +36,6 @@ def _make_scheduler(args): def run_experiments(experiments=None, search_alg=None, scheduler=None, - restore_from_path=None, checkpoint_dir=None, checkpoint_freq=0, with_server=False, @@ -56,10 +54,8 @@ 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. - Defaults to DEFAULT_RESULTS_DIR. + checkpoint_dir (str): Path at which experiment checkpoints are stored + and restored from. 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 @@ -95,12 +91,12 @@ def run_experiments(experiments=None, """ - if restore_from_path: - if not os.path.exists(restore_from_path): - raise ValueError("Provided path invalid: %s" % restore_from_path) - assert not experiments, ( - "Simultaneous starting experiments and restoring not supported.") - runner = TrialRunner.restore(restore_from_path, trial_executor) + if checkpoint_dir and os.path.exists( + os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE)): + if experiments: + logger.warn("Restoring from previous experiment and " + "ignoring given specification.") + runner = TrialRunner.restore(checkpoint_dir, trial_executor) else: if scheduler is None: scheduler = FIFOScheduler() @@ -113,7 +109,7 @@ def run_experiments(experiments=None, runner = TrialRunner( search_alg, scheduler=scheduler, - checkpoint_dir=checkpoint_dir or DEFAULT_RESULTS_DIR, + checkpoint_dir=checkpoint_dir, checkpoint_freq=checkpoint_freq, launch_web_server=with_server, server_port=server_port, From adaaf4348d054e069dfc675bae63bc4b0f0d5460 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Fri, 14 Dec 2018 16:04:36 -0800 Subject: [PATCH 102/172] Atomic Movement --- python/ray/tune/trial_runner.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index 41afe44cfbd16..a6b166069f2c4 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -126,10 +126,12 @@ def checkpoint(self): self.trial_executor.get_checkpoints().values()), "runner": self } - with open(os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE), - "wb") as f: + tmp_file_name = os.path.join(checkpoint_dir, ".tmp_checkpoint") + with open(tmp_file_name, "wb") as f: pickle.dump(runner_state, f) + os.rename(tmp_file_name, os.path.join( + checkpoint_dir, TrialRunner.CKPT_FILE)) return checkpoint_dir @classmethod From 197687373007d1aafc1e58d38bd7599c78178ba2 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Fri, 14 Dec 2018 16:10:08 -0800 Subject: [PATCH 103/172] removed checkpoint freq --- python/ray/tune/experiment.py | 1 - python/ray/tune/ray_trial_executor.py | 5 +++-- python/ray/tune/test/cluster_tests.py | 4 +--- python/ray/tune/test/trial_runner_test.py | 4 ++-- python/ray/tune/trial_runner.py | 22 +++------------------- python/ray/tune/tune.py | 4 ---- 6 files changed, 9 insertions(+), 31 deletions(-) diff --git a/python/ray/tune/experiment.py b/python/ray/tune/experiment.py index 3a4ddc9c7aab8..c1948ffa27e17 100644 --- a/python/ray/tune/experiment.py +++ b/python/ray/tune/experiment.py @@ -71,7 +71,6 @@ class Experiment(object): >>> num_samples=10, >>> local_dir="~/ray_results", >>> upload_dir="s3://your_bucket/path", - >>> checkpoint_freq=10, >>> max_failures=2) """ diff --git a/python/ray/tune/ray_trial_executor.py b/python/ray/tune/ray_trial_executor.py index a214795f53e6c..54888e9775fe3 100644 --- a/python/ray/tune/ray_trial_executor.py +++ b/python/ray/tune/ray_trial_executor.py @@ -19,8 +19,9 @@ class RayTrialExecutor(TrialExecutor): """An implemention of TrialExecutor based on Ray.""" - def __init__(self, queue_trials=False, checkpoint_mode=False): - super(RayTrialExecutor, self).__init__(queue_trials, checkpoint_mode) + def __init__(self, queue_trials=False): + super(RayTrialExecutor, self).__init__( + queue_trials, checkpoint_mode=True) self._running = {} # Since trial resume after paused should not run # trial.train.remote(), thus no more new remote object id generated. diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 3498b0ce434c4..6a6fcb73ea064 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -277,7 +277,7 @@ def test_cluster_down_simple(start_connected_cluster, tmpdir): dirpath = str(tmpdir) runner = TrialRunner( - BasicVariantGenerator(), checkpoint_freq=2, checkpoint_dir=dirpath) + BasicVariantGenerator(), checkpoint_dir=dirpath) kwargs = { "stopping_criterion": { "training_iteration": 2 @@ -334,7 +334,6 @@ def test_cluster_down_full(start_connected_cluster, tmpdir): 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) ray.shutdown() @@ -378,7 +377,6 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): tune.run_experiments( dict(experiment1=kwargs), checkpoint_dir="{checkpoint_dir}", - checkpoint_freq=3, raise_on_failed_trial=False) """.format( redis_address=cluster.redis_address, diff --git a/python/ray/tune/test/trial_runner_test.py b/python/ray/tune/test/trial_runner_test.py index 302b00864de9a..01955a1b03e2c 100644 --- a/python/ray/tune/test/trial_runner_test.py +++ b/python/ray/tune/test/trial_runner_test.py @@ -1563,7 +1563,7 @@ def testTrialSaveRestore(self): default_resources = Resources(cpu=1, gpu=0) runner = TrialRunner( - BasicVariantGenerator(), checkpoint_dir=tmpdir, checkpoint_freq=1) + BasicVariantGenerator(), checkpoint_dir=tmpdir) trials = [ Trial( "__fake", @@ -1627,7 +1627,7 @@ def testTrialNoSave(self): default_resources = Resources(cpu=1, gpu=0) runner = TrialRunner( - BasicVariantGenerator(), checkpoint_dir=tmpdir, checkpoint_freq=1) + BasicVariantGenerator(), checkpoint_dir=tmpdir) trials = [ Trial( "__fake", diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index a6b166069f2c4..98c8d0c8782f0 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -57,7 +57,6 @@ def __init__(self, scheduler=None, launch_web_server=False, checkpoint_dir=None, - checkpoint_freq=0, server_port=TuneServer.DEFAULT_PORT, verbose=True, queue_trials=False, @@ -71,8 +70,6 @@ def __init__(self, launch_web_server (bool): Flag for starting TuneServer checkpoint_dir (str): Path where global checkpoints are stored and restored from. - 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 will not be output. @@ -85,8 +82,7 @@ def __init__(self, self._search_alg = search_alg self._scheduler_alg = scheduler or FIFOScheduler() self.trial_executor = trial_executor or \ - RayTrialExecutor(queue_trials=queue_trials, - checkpoint_mode=checkpoint_freq > 0) + RayTrialExecutor(queue_trials=queue_trials) # For debugging, it may be useful to halt trials after some time has # elapsed. TODO(ekl) consider exposing this in the API. @@ -106,13 +102,6 @@ def __init__(self, self._stop_queue = [] self._trial_checkpoints = {} self._checkpoint_dir = checkpoint_dir - self._checkpoint_freq = checkpoint_freq - if self._checkpoint_freq and not self._checkpoint_dir: - logger.warning("No checkpoint directory specified - turning " - "off checkpointing.") - self._checkpoint_freq = 0 - - assert self._checkpoint_freq >= 0, "checkpoint_freq must be positive." def checkpoint(self): """Saves all trial checkpoints to `self._checkpoint_dir.`""" @@ -164,8 +153,7 @@ def restore(cls, checkpoint_dir, trial_executor=None): runner._scheduler_alg = FIFOScheduler() runner.trial_executor = trial_executor or \ - RayTrialExecutor(queue_trials=runner._queue_trials, - checkpoint_mode=runner._checkpoint_freq > 0) + RayTrialExecutor(queue_trials=runner._queue_trials) logger.info("Adding all trials with checkpoint state.") for ckpt in runner_state["checkpoints"]: @@ -218,11 +206,7 @@ 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 - or self.is_finished()): - self.checkpoint() - + self.checkpoint() self._iteration += 1 if self._server: diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index 9f62ed9d0c755..f0c7627b59fe8 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -37,7 +37,6 @@ def run_experiments(experiments=None, search_alg=None, scheduler=None, checkpoint_dir=None, - checkpoint_freq=0, with_server=False, server_port=TuneServer.DEFAULT_PORT, verbose=True, @@ -56,8 +55,6 @@ def run_experiments(experiments=None, AsyncHyperBand, and HyperBand. checkpoint_dir (str): Path at which experiment checkpoints are stored and restored from. - 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. @@ -110,7 +107,6 @@ def run_experiments(experiments=None, search_alg, scheduler=scheduler, checkpoint_dir=checkpoint_dir, - checkpoint_freq=checkpoint_freq, launch_web_server=with_server, server_port=server_port, verbose=verbose, From c2a578b409aca6bd4a9893ae339929994a81d928 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Fri, 14 Dec 2018 17:32:43 -0800 Subject: [PATCH 104/172] tweaks to update --- python/ray/tune/experiment.py | 4 +++ python/ray/tune/test/cluster_tests.py | 37 ++++++++++++++--------- python/ray/tune/test/trial_runner_test.py | 6 ++-- python/ray/tune/trial_runner.py | 8 +++-- python/ray/tune/tune.py | 20 +++++++++--- 5 files changed, 48 insertions(+), 27 deletions(-) diff --git a/python/ray/tune/experiment.py b/python/ray/tune/experiment.py index 71767f08b9c79..90af0aea78416 100644 --- a/python/ray/tune/experiment.py +++ b/python/ray/tune/experiment.py @@ -124,6 +124,10 @@ def __init__(self, self.name = name self.spec = spec + def is_checkpointable(self): + """Returns whether any trial in experiment will need to checkpoint.""" + return bool(self.checkpoint_freq or self.checkpoint_at_end) + @classmethod def from_json(cls, name, spec): """Generates an Experiment object from JSON. diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 6a6fcb73ea064..e3d28297b3603 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -276,8 +276,7 @@ def test_cluster_down_simple(start_connected_cluster, tmpdir): assert cluster.wait_for_nodes() dirpath = str(tmpdir) - runner = TrialRunner( - BasicVariantGenerator(), checkpoint_dir=dirpath) + runner = TrialRunner(BasicVariantGenerator(), checkpoint_dir=dirpath) kwargs = { "stopping_criterion": { "training_iteration": 2 @@ -319,7 +318,10 @@ def test_cluster_down_full(start_connected_cluster, tmpdir): dirpath = str(tmpdir) exp1_args = dict( - run="__fake", stop=dict(training_iteration=3), checkpoint_freq=1) + run="__fake", + stop=dict(training_iteration=3), + local_dir=dirpath, + checkpoint_freq=1) exp2_args = dict(run="__fake", stop=dict(training_iteration=3)) exp3_args = dict( run="__fake", @@ -330,20 +332,20 @@ def test_cluster_down_full(start_connected_cluster, tmpdir): stop=dict(training_iteration=3), config=dict(mock_error=True), checkpoint_freq=1) + all_experiments = { + "exp1": exp1_args, + "exp2": exp2_args, + "exp3": exp3_args, + "exp4": exp4_args + } - tune.run_experiments( - dict(exp1=exp1_args, exp2=exp2_args, exp3=exp3_args, exp4=exp4_args), - checkpoint_dir=dirpath, - raise_on_failed_trial=False) + tune.run_experiments(all_experiments, raise_on_failed_trial=False) ray.shutdown() cluster.shutdown() cluster = _start_new_cluster() - # Check that last_result.iteration = 1 - runner = TrialRunner.restore(dirpath) - trials = runner.get_trials() - trials = tune.run_experiments(checkpoint_dir=dirpath) + trials = tune.run_experiments(all_experiments, resume=True) assert len(trials) == 2 assert all(t.status in [Trial.TERMINATED, Trial.ERROR] for t in trials) cluster.shutdown() @@ -370,13 +372,12 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): kwargs = dict( run="test", stop=dict(training_iteration=5), + local_dir="{checkpoint_dir}", 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}", + dict(experiment=kwargs), raise_on_failed_trial=False) """.format( redis_address=cluster.redis_address, @@ -411,6 +412,12 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): # Restore properly from checkpoint trials = tune.run_experiments( - checkpoint_dir=dirpath, raise_on_failed_trial=False) + { + "experiment": { + "run": "test" + } + }, + resume=True, + raise_on_failed_trial=False) assert all(t.status == Trial.ERROR for t in trials) cluster.shutdown() diff --git a/python/ray/tune/test/trial_runner_test.py b/python/ray/tune/test/trial_runner_test.py index 7e0a853e3b4b9..f728fd0702013 100644 --- a/python/ray/tune/test/trial_runner_test.py +++ b/python/ray/tune/test/trial_runner_test.py @@ -1642,8 +1642,7 @@ def testTrialSaveRestore(self): tmpdir = tempfile.mkdtemp() default_resources = Resources(cpu=1, gpu=0) - runner = TrialRunner( - BasicVariantGenerator(), checkpoint_dir=tmpdir) + runner = TrialRunner(BasicVariantGenerator(), checkpoint_dir=tmpdir) trials = [ Trial( "__fake", @@ -1706,8 +1705,7 @@ def testTrialNoSave(self): tmpdir = tempfile.mkdtemp() default_resources = Resources(cpu=1, gpu=0) - runner = TrialRunner( - BasicVariantGenerator(), checkpoint_dir=tmpdir) + runner = TrialRunner(BasicVariantGenerator(), checkpoint_dir=tmpdir) trials = [ Trial( "__fake", diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index 98c8d0c8782f0..b5a97df2f6479 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -104,7 +104,9 @@ def __init__(self, self._checkpoint_dir = checkpoint_dir def checkpoint(self): - """Saves all trial checkpoints to `self._checkpoint_dir.`""" + """Saves execution state to `self._checkpoint_dir` if provided.""" + if not self._checkpoint_dir: + return checkpoint_dir = self._checkpoint_dir if not os.path.exists(checkpoint_dir): logger.debug("Checkpoint directory newly created.") @@ -119,8 +121,8 @@ def checkpoint(self): with open(tmp_file_name, "wb") as f: pickle.dump(runner_state, f) - os.rename(tmp_file_name, os.path.join( - checkpoint_dir, TrialRunner.CKPT_FILE)) + os.rename(tmp_file_name, + os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE)) return checkpoint_dir @classmethod diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index f0c7627b59fe8..6eb446d158234 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -7,6 +7,7 @@ import time from ray.tune.error import TuneError +from ray.tune.experiment import convert_to_experiment_list from ray.tune.suggest import BasicVariantGenerator from ray.tune.trial import Trial, DEBUG_PRINT_INTERVAL from ray.tune.log_sync import wait_for_log_sync @@ -33,13 +34,18 @@ def _make_scheduler(args): args.scheduler, _SCHEDULERS.keys())) +def _find_checkpoint_dir(exp_list): + checkpointable_expts = [exp for exp in exp_list if exp.is_checkpointable()] + return checkpointable_expts[0].spec["local_dir"] + + def run_experiments(experiments=None, search_alg=None, scheduler=None, - checkpoint_dir=None, with_server=False, server_port=TuneServer.DEFAULT_PORT, verbose=True, + resume=False, queue_trials=False, trial_executor=None, raise_on_failed_trial=True): @@ -59,6 +65,9 @@ def run_experiments(experiments=None, using the Client API. server_port (int): Port number for launching TuneServer. verbose (bool): How much output should be printed for each trial. + resume (bool): To resume from full experiment checkpoint. Only the + first checkpointable experiment local_dir is checked. + If checkpoint exists, the experiment will resume from there. queue_trials (bool): Whether to queue trials when the cluster does not currently have enough resources to launch one. This should be set to True when running on an autoscaling cluster to enable @@ -87,12 +96,13 @@ def run_experiments(experiments=None, List of Trial objects, holding data for each executed trial. """ + experiments = convert_to_experiment_list(experiments) + checkpoint_dir = _find_checkpoint_dir(experiments) - if checkpoint_dir and os.path.exists( + if resume and os.path.exists( os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE)): - if experiments: - logger.warn("Restoring from previous experiment and " - "ignoring given specification.") + logger.warn("Restoring from previous experiment and " + "ignoring any new changes to specification.") runner = TrialRunner.restore(checkpoint_dir, trial_executor) else: if scheduler is None: From 60653c17364fd61418a7aaf596fa827f0ed074a9 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Fri, 14 Dec 2018 18:02:05 -0800 Subject: [PATCH 105/172] move sync function --- python/ray/tune/experiment.py | 2 -- python/ray/tune/log_sync.py | 28 ++++++++++++++-------------- python/ray/tune/logger.py | 3 ++- 3 files changed, 16 insertions(+), 17 deletions(-) diff --git a/python/ray/tune/experiment.py b/python/ray/tune/experiment.py index 90af0aea78416..fe2e27965b9f0 100644 --- a/python/ray/tune/experiment.py +++ b/python/ray/tune/experiment.py @@ -8,7 +8,6 @@ import types from ray.tune.error import TuneError -from ray.tune.log_sync import validate_sync_function from ray.tune.registry import register_trainable from ray.tune.result import DEFAULT_RESULTS_DIR @@ -101,7 +100,6 @@ def __init__(self, checkpoint_at_end=False, max_failures=3, restore=None): - validate_sync_function(sync_function) if sync_function: assert upload_dir, "Need `upload_dir` if sync_function given." spec = { diff --git a/python/ray/tune/log_sync.py b/python/ray/tune/log_sync.py index 2046165c0129d..70bb80df4df40 100644 --- a/python/ray/tune/log_sync.py +++ b/python/ray/tune/log_sync.py @@ -30,6 +30,20 @@ ALLOWED_REMOTE_PREFIXES = (S3_PREFIX, GCS_PREFIX) +def validate_sync_function(sync_function): + if sync_function is None: + return + elif isinstance(sync_function, str): + assert "{remote_dir}" in sync_function, ( + "Sync template missing '{remote_dir}'.") + assert "{local_dir}" in sync_function, ( + "Sync template missing '{local_dir}'.") + elif not (isinstance(sync_function, types.FunctionType) + or isinstance(sync_function, tune_function)): + raise ValueError("Sync function {} must be string or function".format( + sync_function)) + + def get_syncer(local_dir, remote_dir=None, sync_function=None): if remote_dir: if not sync_function and not any( @@ -65,20 +79,6 @@ def wait_for_log_sync(): syncer.wait() -def validate_sync_function(sync_function): - if sync_function is None: - return - elif isinstance(sync_function, str): - assert "{remote_dir}" in sync_function, ( - "Sync template missing '{remote_dir}'.") - assert "{local_dir}" in sync_function, ( - "Sync template missing '{local_dir}'.") - elif not (isinstance(sync_function, types.FunctionType) - or isinstance(sync_function, tune_function)): - raise ValueError("Sync function {} must be string or function".format( - sync_function)) - - class _LogSyncer(object): """Log syncer for tune. diff --git a/python/ray/tune/logger.py b/python/ray/tune/logger.py index c8b41f591348f..b17b4c104b27c 100644 --- a/python/ray/tune/logger.py +++ b/python/ray/tune/logger.py @@ -10,7 +10,7 @@ import yaml import ray.cloudpickle as cloudpickle -from ray.tune.log_sync import get_syncer +from ray.tune.log_sync import get_syncer, validate_sync_function from ray.tune.result import NODE_IP, TRAINING_ITERATION, TIME_TOTAL_S, \ TIMESTEPS_TOTAL @@ -82,6 +82,7 @@ def __init__(self, upload_uri=None, custom_loggers=None, sync_function=None): + validate_sync_function(sync_function) self._logger_list = [_JsonLogger, _TFLogger, _VisKitLogger] self._sync_function = sync_function if custom_loggers: From c0d6db5b145787ea9d975558bf846a27ce56e59c Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Fri, 14 Dec 2018 20:51:25 -0800 Subject: [PATCH 106/172] fix --- python/ray/tune/experiment.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/python/ray/tune/experiment.py b/python/ray/tune/experiment.py index fe2e27965b9f0..8bb4de0ed03b0 100644 --- a/python/ray/tune/experiment.py +++ b/python/ray/tune/experiment.py @@ -124,7 +124,8 @@ def __init__(self, def is_checkpointable(self): """Returns whether any trial in experiment will need to checkpoint.""" - return bool(self.checkpoint_freq or self.checkpoint_at_end) + return bool(self.spec["checkpoint_freq"] + or self.spec["checkpoint_at_end"]) @classmethod def from_json(cls, name, spec): From 96c9f7048b4a9b97732eff4a535e5537bc0dc4ed Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Sat, 15 Dec 2018 09:23:26 -0800 Subject: [PATCH 107/172] fixup --- python/ray/tune/experiment.py | 2 +- python/ray/tune/log_sync.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/python/ray/tune/experiment.py b/python/ray/tune/experiment.py index 8bb4de0ed03b0..e432a82a7ecd4 100644 --- a/python/ray/tune/experiment.py +++ b/python/ray/tune/experiment.py @@ -112,7 +112,7 @@ def __init__(self, "upload_dir": upload_dir or "", # argparse converts None to "null" "trial_name_creator": trial_name_creator, "custom_loggers": custom_loggers, - "sync_function": sync_function or "", # See `upload_dir`. + "sync_function": sync_function, "checkpoint_freq": checkpoint_freq, "checkpoint_at_end": checkpoint_at_end, "max_failures": max_failures, diff --git a/python/ray/tune/log_sync.py b/python/ray/tune/log_sync.py index 70bb80df4df40..8727448041f9e 100644 --- a/python/ray/tune/log_sync.py +++ b/python/ray/tune/log_sync.py @@ -35,7 +35,7 @@ def validate_sync_function(sync_function): return elif isinstance(sync_function, str): assert "{remote_dir}" in sync_function, ( - "Sync template missing '{remote_dir}'.") + "Sync template missing '{remote_dir}'." ) assert "{local_dir}" in sync_function, ( "Sync template missing '{local_dir}'.") elif not (isinstance(sync_function, types.FunctionType) From 2afe2917054e7427128b241d80614434ad143a0f Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Sat, 15 Dec 2018 09:32:12 -0800 Subject: [PATCH 108/172] small modifications --- python/ray/tune/tune.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index 6eb446d158234..982c68925f410 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -36,7 +36,10 @@ def _make_scheduler(args): def _find_checkpoint_dir(exp_list): checkpointable_expts = [exp for exp in exp_list if exp.is_checkpointable()] - return checkpointable_expts[0].spec["local_dir"] + if checkpointable_expts: + return checkpointable_expts[0].spec["local_dir"] + else: + return None def run_experiments(experiments=None, @@ -99,7 +102,7 @@ def run_experiments(experiments=None, experiments = convert_to_experiment_list(experiments) checkpoint_dir = _find_checkpoint_dir(experiments) - if resume and os.path.exists( + if resume and checkpoint_dir and os.path.exists( os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE)): logger.warn("Restoring from previous experiment and " "ignoring any new changes to specification.") From a5647dbabc119d52c8c2e503f71e32ae8144c807 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Sat, 15 Dec 2018 09:36:47 -0800 Subject: [PATCH 109/172] fix --- python/ray/tune/test/cluster_tests.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index e3d28297b3603..d999a98fb06d2 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -411,7 +411,7 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): assert trials[0].status == Trial.PENDING # Restore properly from checkpoint - trials = tune.run_experiments( + trials2 = tune.run_experiments( { "experiment": { "run": "test" @@ -419,5 +419,6 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): }, resume=True, raise_on_failed_trial=False) - assert all(t.status == Trial.ERROR for t in trials) + assert all(t.status == Trial.ERROR for t in trials2) + assert set(t.trial_id for t in trials2) == set(t.trial_id for t in trials) cluster.shutdown() From f17a648bcf8786a3d18665989f2f3865c3a74635 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Sat, 15 Dec 2018 09:37:34 -0800 Subject: [PATCH 110/172] fix --- python/ray/tune/test/cluster_tests.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index d999a98fb06d2..67fd67213ce8c 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -415,6 +415,7 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): { "experiment": { "run": "test" + "local_dir": dirpath } }, resume=True, From 52ac11b2fe36ee518089b56c7f04dca99394018c Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 17 Dec 2018 22:53:15 -0800 Subject: [PATCH 111/172] fix --- python/ray/tune/logger.py | 3 +-- python/ray/tune/trial.py | 2 ++ 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/python/ray/tune/logger.py b/python/ray/tune/logger.py index b17b4c104b27c..c8b41f591348f 100644 --- a/python/ray/tune/logger.py +++ b/python/ray/tune/logger.py @@ -10,7 +10,7 @@ import yaml import ray.cloudpickle as cloudpickle -from ray.tune.log_sync import get_syncer, validate_sync_function +from ray.tune.log_sync import get_syncer from ray.tune.result import NODE_IP, TRAINING_ITERATION, TIME_TOTAL_S, \ TIMESTEPS_TOTAL @@ -82,7 +82,6 @@ def __init__(self, upload_uri=None, custom_loggers=None, sync_function=None): - validate_sync_function(sync_function) self._logger_list = [_JsonLogger, _TFLogger, _VisKitLogger] self._sync_function = sync_function if custom_loggers: diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index 6a047178b8ae1..b72fcc15df8bc 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -12,6 +12,7 @@ import ray from ray.tune import TuneError +from ray.tune.log_sync import validate_sync_function from ray.tune.logger import pretty_print, UnifiedLogger # NOTE(rkn): We import ray.tune.registry here instead of importing the names we # need because there are cyclic imports that may cause specific names to not @@ -152,6 +153,7 @@ def __init__(self, self.trial_name_creator = trial_name_creator self.custom_loggers = custom_loggers self.sync_function = sync_function + validate_sync_function(sync_function) self.verbose = True self.max_failures = max_failures From 8a57a5b36dec8a951ff0e1b191402e777b48073a Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 17 Dec 2018 23:15:46 -0800 Subject: [PATCH 112/172] fix assumption --- python/ray/tune/trial_runner.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index b5a97df2f6479..98cd6954f5680 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -500,9 +500,8 @@ def __getstate__(self): state["trial_executor"] = None state["_search_alg"] = None - if not isinstance(state["_scheduler_alg"], FIFOScheduler): - # TODO(rliaw): Remove this once component FT is implemented - state["_scheduler_alg"] = None + # TODO(rliaw): Remove this once component FT is implemented + state["_scheduler_alg"] = None if state["_server"]: state["_launch_web_server"] = True From 530d5af1541e4368e68aa45db4199f6b0e4ec35a Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 17 Dec 2018 23:33:32 -0800 Subject: [PATCH 113/172] better error message --- python/ray/tune/experiment.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/python/ray/tune/experiment.py b/python/ray/tune/experiment.py index e432a82a7ecd4..a3b35bb86eccd 100644 --- a/python/ray/tune/experiment.py +++ b/python/ray/tune/experiment.py @@ -157,8 +157,9 @@ def from_json(cls, name, spec): run_value = spec.pop("run") try: exp = cls(name, run_value, **spec) - except TypeError: - raise TuneError("Improper argument from JSON: {}.".format(spec)) + except TypeError as exc: + logger.exception("Improper argument from JSON.") + raise exc return exp def _register_if_needed(self, run_object): From 758ed3059e4f50ddecf47b8a2bfa192c88c15140 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 17 Dec 2018 23:34:18 -0800 Subject: [PATCH 114/172] fix registration --- python/ray/tune/trial.py | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index b72fcc15df8bc..3ab682729d74c 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -134,12 +134,8 @@ def __init__(self, The args here take the same meaning as the command line flags defined in ray.tune.config_parser. """ - if not has_trainable(trainable_name): - # Make sure rllib agents are registered - from ray import rllib # noqa: F401 - if not has_trainable(trainable_name): - raise TuneError("Unknown trainable: " + trainable_name) + Trial._registration_check(trainable_name) # Trial config self.trainable_name = trainable_name self.config = config or {} @@ -172,6 +168,14 @@ def __init__(self, self.error_file = None self.num_failures = 0 + @classmethod + def _registration_check(cls, trainable_name): + if not has_trainable(trainable_name): + # Make sure rllib agents are registered + from ray import rllib # noqa: F401 + if not has_trainable(trainable_name): + raise TuneError("Unknown trainable: " + trainable_name) + @classmethod def generate_id(cls): return binary_to_hex(random_string())[:8] @@ -369,5 +373,6 @@ def __getstate__(self): def __setstate__(self, state): logger_started = state.pop("_logger_started") self.__dict__.update(state) + Trial._registration_check(self.__dict__["trainable_name"]) if logger_started: self.init_logger() From 1ff48142226dafcb9c3d2b2c76619e46d8d1c2ec Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 17 Dec 2018 23:46:10 -0800 Subject: [PATCH 115/172] ok fix --- python/ray/tune/trial_runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index 98cd6954f5680..df0c7b4da2798 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -110,8 +110,8 @@ def checkpoint(self): checkpoint_dir = self._checkpoint_dir if not os.path.exists(checkpoint_dir): logger.debug("Checkpoint directory newly created.") + logger.warning("Search Algorithm and Scheduler not checkpointed.") os.makedirs(checkpoint_dir) - logger.warning("Search Algorithm and Scheduler not checkpointed.") runner_state = { "checkpoints": list( self.trial_executor.get_checkpoints().values()), From 1052d3bfe30914fa281ae7c11d57fad81cdc207b Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 18 Dec 2018 00:00:24 -0800 Subject: [PATCH 116/172] fix add test --- python/ray/tune/log_sync.py | 2 +- python/ray/tune/test/cluster_tests.py | 96 ++++++++++++++++++++------- 2 files changed, 73 insertions(+), 25 deletions(-) diff --git a/python/ray/tune/log_sync.py b/python/ray/tune/log_sync.py index 8727448041f9e..70bb80df4df40 100644 --- a/python/ray/tune/log_sync.py +++ b/python/ray/tune/log_sync.py @@ -35,7 +35,7 @@ def validate_sync_function(sync_function): return elif isinstance(sync_function, str): assert "{remote_dir}" in sync_function, ( - "Sync template missing '{remote_dir}'." ) + "Sync template missing '{remote_dir}'.") assert "{local_dir}" in sync_function, ( "Sync template missing '{local_dir}'.") elif not (isinstance(sync_function, types.FunctionType) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 67fd67213ce8c..04150a2f63273 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -23,27 +23,24 @@ from ray.tune.suggest import BasicVariantGenerator -def register_fail_trainable(): - class _Fail(tune.Trainable): - """Fails on the 4th iteration.""" +class _Fail(tune.Trainable): + """Fails on the 4th iteration.""" - def _setup(self, config): - self.state = {"hi": 0} + def _setup(self, config): + self.state = {"hi": 0} - def _train(self): - self.state["hi"] += 1 - time.sleep(0.5) - if self.state["hi"] >= 4: - assert False - return {} + def _train(self): + self.state["hi"] += 1 + time.sleep(0.5) + if self.state["hi"] >= 4: + assert False + return {} - def _save(self, path): - return self.state + def _save(self, path): + return self.state - def _restore(self, state): - self.state = state - - tune.register_trainable("test", _Fail) + def _restore(self, state): + self.state = state def _start_new_cluster(): @@ -351,6 +348,59 @@ def test_cluster_down_full(start_connected_cluster, tmpdir): cluster.shutdown() +def test_cluster_rllib_restore(start_connected_cluster, tmpdir): + cluster = start_connected_cluster + dirpath = str(tmpdir) + script = """ +import time +import ray +from ray import tune + +ray.init(redis_address="{redis_address}") + +kwargs = dict( + run="PG", + env="CartPole-v1", + stop=dict(training_iteration=10), + local_dir="{checkpoint_dir}", + checkpoint_freq=1, + max_failures=1) + +tune.run_experiments( + dict(experiment=kwargs), + raise_on_failed_trial=False) +""".format( + redis_address=cluster.redis_address, checkpoint_dir=dirpath) + 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, TrialRunner.CKPT_FILE)): + # Inspect the internal trialrunner + runner = TrialRunner.restore(dirpath) + trials = runner.get_trials() + last_res = trials[0].last_result + if last_res is not None and last_res["training_iteration"]: + break + time.sleep(0.2) + + ray.shutdown() + cluster.shutdown() + cluster = _start_new_cluster() + + # Restore properly from checkpoint + trials2 = tune.run_experiments( + { + "experiment": { + "run": "PG", + "local_dir": dirpath + } + }, resume=True) + assert all(t.status == Trial.TERMINATED for t in trials2) + cluster.shutdown() + + def test_cluster_interrupt(start_connected_cluster, tmpdir): """Tests run_experiment on cluster shutdown even with atypical trial. @@ -366,11 +416,10 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): ray.init(redis_address="{redis_address}") -{register_trainable_fn} -{run_register_trainable_fn}() +{fail_class} kwargs = dict( - run="test", + run="{fail_class_name}", stop=dict(training_iteration=5), local_dir="{checkpoint_dir}", checkpoint_freq=1, @@ -382,8 +431,8 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): """.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__) + fail_class=inspect.getsource(_Fail), + fail_class_name=_Fail.__name__) run_string_as_driver_nonblocking(script) # Wait until the right checkpoint is saved. @@ -402,7 +451,6 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): ray.shutdown() cluster.shutdown() cluster = _start_new_cluster() - register_fail_trainable() # Inspect the internal trialrunner just in case runner = TrialRunner.restore(dirpath) @@ -414,7 +462,7 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): trials2 = tune.run_experiments( { "experiment": { - "run": "test" + "run": _Fail, "local_dir": dirpath } }, From e0963e56eb5e0c11d5cc80e73760b37f2d158947 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 18 Dec 2018 00:04:19 -0800 Subject: [PATCH 117/172] cloudpickle --- python/ray/tune/trial_executor.py | 4 ++-- python/ray/tune/trial_runner.py | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/python/ray/tune/trial_executor.py b/python/ray/tune/trial_executor.py index 5ff39e78b62cd..246d1ff15f207 100644 --- a/python/ray/tune/trial_executor.py +++ b/python/ray/tune/trial_executor.py @@ -4,8 +4,8 @@ from __future__ import print_function import logging -import pickle +import ray.cloudpickle as cloudpickle from ray.tune.trial import Trial, Checkpoint logger = logging.getLogger(__name__) @@ -58,7 +58,7 @@ def try_checkpoint_metadata(self, trial): return try: logger.debug("Saving trial metadata.") - metadata = pickle.dumps(trial) + metadata = cloudpickle.dumps(trial) self._checkpoints[trial.trial_id] = metadata except ValueError: logger.exception("Error checkpointing trial metadata.") diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index df0c7b4da2798..10f6c63b35344 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -5,11 +5,11 @@ import collections import logging import os -import pickle import re import time import traceback +import ray.cloudpickle as cloudpickle from ray.tune import TuneError from ray.tune.ray_trial_executor import RayTrialExecutor from ray.tune.result import TIME_THIS_ITER_S @@ -119,7 +119,7 @@ def checkpoint(self): } tmp_file_name = os.path.join(checkpoint_dir, ".tmp_checkpoint") with open(tmp_file_name, "wb") as f: - pickle.dump(runner_state, f) + cloudpickle.dump(runner_state, f) os.rename(tmp_file_name, os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE)) @@ -142,7 +142,7 @@ def restore(cls, checkpoint_dir, trial_executor=None): """ with open(os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE), "rb") as f: - runner_state = pickle.load(f) + runner_state = cloudpickle.load(f) logger.warning( "Tune recovery is still experimental. " @@ -159,7 +159,7 @@ def restore(cls, checkpoint_dir, trial_executor=None): logger.info("Adding all trials with checkpoint state.") for ckpt in runner_state["checkpoints"]: - trial = pickle.loads(ckpt) + trial = cloudpickle.loads(ckpt) runner.add_trial(trial) return runner From 5358f0c04f2d914d8bdbfc63f747c6b5f86224bd Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 18 Dec 2018 00:25:19 -0800 Subject: [PATCH 118/172] classmethod --- python/ray/tune/experiment.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/python/ray/tune/experiment.py b/python/ray/tune/experiment.py index e432a82a7ecd4..a9c779c8ef92c 100644 --- a/python/ray/tune/experiment.py +++ b/python/ray/tune/experiment.py @@ -103,7 +103,7 @@ def __init__(self, if sync_function: assert upload_dir, "Need `upload_dir` if sync_function given." spec = { - "run": self._register_if_needed(run), + "run": Experiment._register_if_needed(run), "stop": stop or {}, "config": config or {}, "trial_resources": trial_resources, @@ -161,7 +161,8 @@ def from_json(cls, name, spec): raise TuneError("Improper argument from JSON: {}.".format(spec)) return exp - def _register_if_needed(self, run_object): + @classmethod + def _register_if_needed(cls, run_object): """Registers Trainable or Function at runtime. Assumes already registered if run_object is a string. Does not From 61034c903186ce8fe79ec5d3414004c71ddcb20d Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 18 Dec 2018 00:34:00 -0800 Subject: [PATCH 119/172] fix --- python/ray/tune/test/cluster_tests.py | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 04150a2f63273..4d6eecb57307e 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -18,6 +18,7 @@ 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.experiment import Experiment from ray.tune.trial import Trial from ray.tune.trial_runner import TrialRunner from ray.tune.suggest import BasicVariantGenerator @@ -416,10 +417,10 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): ray.init(redis_address="{redis_address}") -{fail_class} +{fail_class_code} kwargs = dict( - run="{fail_class_name}", + run={fail_class}, stop=dict(training_iteration=5), local_dir="{checkpoint_dir}", checkpoint_freq=1, @@ -431,14 +432,14 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): """.format( redis_address=cluster.redis_address, checkpoint_dir=dirpath, - fail_class=inspect.getsource(_Fail), - fail_class_name=_Fail.__name__) + fail_class_code=inspect.getsource(_Fail), + fail_class=_Fail.__name__) 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): + for i in range(50): if os.path.exists(os.path.join(dirpath, TrialRunner.CKPT_FILE)): # Inspect the internal trialrunner runner = TrialRunner.restore(dirpath) @@ -448,9 +449,13 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): break time.sleep(0.2) + if not os.path.exists(os.path.join(dirpath, TrialRunner.CKPT_FILE)): + raise RuntimeError("Checkpoint file didn't appear.") + ray.shutdown() cluster.shutdown() cluster = _start_new_cluster() + Experiment._register_if_needed(_Fail) # Inspect the internal trialrunner just in case runner = TrialRunner.restore(dirpath) @@ -463,7 +468,8 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): { "experiment": { "run": _Fail, - "local_dir": dirpath + "local_dir": dirpath, + "checkpoint_freq": 1 } }, resume=True, From b301963422040b334146a194d033a788d06e3d0b Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 18 Dec 2018 01:02:00 -0800 Subject: [PATCH 120/172] lint... --- 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 4d6eecb57307e..24e20aef465ee 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -474,6 +474,6 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): }, resume=True, raise_on_failed_trial=False) - assert all(t.status == Trial.ERROR for t in trials2) - assert set(t.trial_id for t in trials2) == set(t.trial_id for t in trials) + assert all([t.status == Trial.ERROR for t in trials2]) + assert set([t.trial_id for t in trials2]) == set([t.trial_id for t in trials]) cluster.shutdown() From 82078465d2a6294e7ca21c3413171ef60d70f1b8 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 18 Dec 2018 01:14:48 -0800 Subject: [PATCH 121/172] Revert "better error message" This reverts commit 530d5af1541e4368e68aa45db4199f6b0e4ec35a. --- python/ray/tune/experiment.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/python/ray/tune/experiment.py b/python/ray/tune/experiment.py index 2066b538e7399..a9c779c8ef92c 100644 --- a/python/ray/tune/experiment.py +++ b/python/ray/tune/experiment.py @@ -157,9 +157,8 @@ def from_json(cls, name, spec): run_value = spec.pop("run") try: exp = cls(name, run_value, **spec) - except TypeError as exc: - logger.exception("Improper argument from JSON.") - raise exc + except TypeError: + raise TuneError("Improper argument from JSON: {}.".format(spec)) return exp @classmethod From c6ec396982dda7d5bfd53447c9a1ab6d3305e700 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 18 Dec 2018 01:30:09 -0800 Subject: [PATCH 122/172] comment --- python/ray/tune/tune.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index 982c68925f410..f25d0eafef513 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -99,6 +99,8 @@ def run_experiments(experiments=None, List of Trial objects, holding data for each executed trial. """ + # This is important here to schematize the experiments + # and also does implicit registration. experiments = convert_to_experiment_list(experiments) checkpoint_dir = _find_checkpoint_dir(experiments) From 0db58a5ff3e27bc7977e17e5220c8ed25e1843c3 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 18 Dec 2018 02:13:12 -0800 Subject: [PATCH 123/172] better handling --- python/ray/tune/trial.py | 17 +++++++++++++---- python/ray/tune/trial_runner.py | 4 ++-- 2 files changed, 15 insertions(+), 6 deletions(-) diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index 3ab682729d74c..abe0b8191058f 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -155,6 +155,7 @@ def __init__(self, # Local trial state that is updated during the run self.last_result = None + self.last_update_time = -float("inf") self.checkpoint_freq = checkpoint_freq self.checkpoint_at_end = checkpoint_at_end self._checkpoint = Checkpoint( @@ -186,10 +187,15 @@ def init_logger(self): if not self.result_logger: if not os.path.exists(self.local_dir): os.makedirs(self.local_dir) - self.logdir = tempfile.mkdtemp( - prefix="{}_{}".format( - str(self)[:MAX_LEN_IDENTIFIER], date_str()), - dir=self.local_dir) + + if not self.logdir: + self.logdir = tempfile.mkdtemp( + prefix="{}_{}".format( + str(self)[:MAX_LEN_IDENTIFIER], date_str()), + dir=self.local_dir) + elif not os.path.exists(self.logdir): + os.makedirs(self.local_dir) + self.result_logger = UnifiedLogger( self.config, self.logdir, @@ -249,6 +255,8 @@ def progress_string(self): return self._status_string() def location_string(hostname, pid): + if self.status == Trial.PENDING: + return "" if hostname == os.uname()[1]: return 'pid={}'.format(pid) else: @@ -313,6 +321,7 @@ def update_last_result(self, result, terminate=False): pretty_print(result).replace("\n", "\n "))) self.last_debug = time.time() self.last_result = result + self.last_update_time = time.time() self.result_logger.on_result(self.last_result) def _get_trainable_cls(self): diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index 10f6c63b35344..dd020219f6d22 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -158,8 +158,8 @@ def restore(cls, checkpoint_dir, trial_executor=None): RayTrialExecutor(queue_trials=runner._queue_trials) logger.info("Adding all trials with checkpoint state.") - for ckpt in runner_state["checkpoints"]: - trial = cloudpickle.loads(ckpt) + trials = [cloudpickle.loads(cp) for cp in runner_state["checkpoints"]] + for trial in sorted(trials, key=lambda t: t.last_update_time, reverse=True): runner.add_trial(trial) return runner From 4d75a38434165cda4e8bec2567bce61f681948e9 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 18 Dec 2018 02:17:08 -0800 Subject: [PATCH 124/172] nit --- python/ray/tune/test/cluster_tests.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 24e20aef465ee..fb80f1fec81b6 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -389,6 +389,7 @@ def test_cluster_rllib_restore(start_connected_cluster, tmpdir): ray.shutdown() cluster.shutdown() cluster = _start_new_cluster() + cluster.wait_for_nodes() # Restore properly from checkpoint trials2 = tune.run_experiments( From da216863692739c19546d86f284a3942e24eab03 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 18 Dec 2018 02:38:47 -0800 Subject: [PATCH 125/172] a little confusing --- python/ray/tune/trial.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index abe0b8191058f..32f02e24bb679 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -255,8 +255,6 @@ def progress_string(self): return self._status_string() def location_string(hostname, pid): - if self.status == Trial.PENDING: - return "" if hostname == os.uname()[1]: return 'pid={}'.format(pid) else: From 386a9fa9cea52be920515dee11622c7ec4df2958 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 18 Dec 2018 02:42:54 -0800 Subject: [PATCH 126/172] lint --- python/ray/tune/test/cluster_tests.py | 3 ++- python/ray/tune/trial_runner.py | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index fb80f1fec81b6..04001489e4c90 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -476,5 +476,6 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): resume=True, raise_on_failed_trial=False) assert all([t.status == Trial.ERROR for t in trials2]) - assert set([t.trial_id for t in trials2]) == set([t.trial_id for t in trials]) + assert set([t.trial_id for t in trials2]) == set( + [t.trial_id for t in trials]) cluster.shutdown() diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index dd020219f6d22..5d88aa8bc44b6 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -159,7 +159,8 @@ def restore(cls, checkpoint_dir, trial_executor=None): logger.info("Adding all trials with checkpoint state.") trials = [cloudpickle.loads(cp) for cp in runner_state["checkpoints"]] - for trial in sorted(trials, key=lambda t: t.last_update_time, reverse=True): + for trial in sorted( + trials, key=lambda t: t.last_update_time, reverse=True): runner.add_trial(trial) return runner From 9c1cdb9e1c2b045dc564055cd86fe238de06808e Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 18 Dec 2018 12:41:32 -0800 Subject: [PATCH 127/172] fixtest --- python/ray/tune/test/cluster_tests.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 04001489e4c90..e223eb01c9d26 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -376,7 +376,7 @@ def test_cluster_rllib_restore(start_connected_cluster, tmpdir): # 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): + for i in range(50): if os.path.exists(os.path.join(dirpath, TrialRunner.CKPT_FILE)): # Inspect the internal trialrunner runner = TrialRunner.restore(dirpath) @@ -386,6 +386,9 @@ def test_cluster_rllib_restore(start_connected_cluster, tmpdir): break time.sleep(0.2) + if not os.path.exists(os.path.join(dirpath, TrialRunner.CKPT_FILE)): + raise RuntimeError("Checkpoint file didn't appear.") + ray.shutdown() cluster.shutdown() cluster = _start_new_cluster() @@ -396,6 +399,7 @@ def test_cluster_rllib_restore(start_connected_cluster, tmpdir): { "experiment": { "run": "PG", + "checkpoint_freq": 1, "local_dir": dirpath } }, resume=True) From 5b69662939b7be582915eb9d72b4b35b6fd5352a Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 19 Dec 2018 23:15:49 -0800 Subject: [PATCH 128/172] fix mess --- python/ray/tune/test/cluster_tests.py | 13 ++++++++--- python/ray/tune/trial_runner.py | 14 +++++++----- python/ray/tune/tune.py | 31 ++++++++++++++++++++++----- 3 files changed, 45 insertions(+), 13 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index e223eb01c9d26..542c0a8f2203d 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -337,13 +337,15 @@ def test_cluster_down_full(start_connected_cluster, tmpdir): "exp4": exp4_args } - tune.run_experiments(all_experiments, raise_on_failed_trial=False) + tune.run_experiments( + all_experiments, checkpoint_mode=True, raise_on_failed_trial=False) ray.shutdown() cluster.shutdown() cluster = _start_new_cluster() - trials = tune.run_experiments(all_experiments, resume=True) + trials = tune.run_experiments( + all_experiments, checkpoint_mode=True, resume=True) assert len(trials) == 2 assert all(t.status in [Trial.TERMINATED, Trial.ERROR] for t in trials) cluster.shutdown() @@ -369,6 +371,7 @@ def test_cluster_rllib_restore(start_connected_cluster, tmpdir): tune.run_experiments( dict(experiment=kwargs), + checkpoint_mode=True, raise_on_failed_trial=False) """.format( redis_address=cluster.redis_address, checkpoint_dir=dirpath) @@ -402,7 +405,9 @@ def test_cluster_rllib_restore(start_connected_cluster, tmpdir): "checkpoint_freq": 1, "local_dir": dirpath } - }, resume=True) + }, + checkpoint_mode=True, + resume=True) assert all(t.status == Trial.TERMINATED for t in trials2) cluster.shutdown() @@ -433,6 +438,7 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): tune.run_experiments( dict(experiment=kwargs), + checkpoint_mode=True, raise_on_failed_trial=False) """.format( redis_address=cluster.redis_address, @@ -478,6 +484,7 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): } }, resume=True, + checkpoint_mode=True, raise_on_failed_trial=False) assert all([t.status == Trial.ERROR for t in trials2]) assert set([t.trial_id for t in trials2]) == set( diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index 5d88aa8bc44b6..a7afd889de448 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -57,6 +57,7 @@ def __init__(self, scheduler=None, launch_web_server=False, checkpoint_dir=None, + checkpoint_mode=False, server_port=TuneServer.DEFAULT_PORT, verbose=True, queue_trials=False, @@ -70,6 +71,8 @@ def __init__(self, launch_web_server (bool): Flag for starting TuneServer checkpoint_dir (str): Path where global checkpoints are stored and restored from. + checkpoint_mode: Turns on checkpointing for full Tune experiment. + Currently defaults to False. server_port (int): Port number for launching TuneServer verbose (bool): Flag for verbosity. If False, trial results will not be output. @@ -102,10 +105,11 @@ def __init__(self, self._stop_queue = [] self._trial_checkpoints = {} self._checkpoint_dir = checkpoint_dir + self._checkpoint_mode = checkpoint_mode def checkpoint(self): """Saves execution state to `self._checkpoint_dir` if provided.""" - if not self._checkpoint_dir: + if not self._checkpoint_mode or not self._checkpoint_dir: return checkpoint_dir = self._checkpoint_dir if not os.path.exists(checkpoint_dir): @@ -241,7 +245,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._checkpoint_trial_if_needed(trial) self._trials.append(trial) def debug_string(self, max_debug=MAX_DEBUG_TRIALS): @@ -356,14 +360,14 @@ def _process_events(self): result, terminate=(decision == TrialScheduler.STOP)) if decision == TrialScheduler.CONTINUE: - self._checkpoint_if_needed(trial) + self._checkpoint_trial_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) + self._checkpoint_trial_if_needed(trial) self.trial_executor.stop_trial(trial) else: assert False, "Invalid scheduling decision: {}".format( @@ -381,7 +385,7 @@ def _process_events(self): self.trial_executor.stop_trial( trial, error=True, error_msg=error_msg) - def _checkpoint_if_needed(self, trial): + def _checkpoint_trial_if_needed(self, trial): """Checkpoints trial based off trial.last_result.""" if trial.should_checkpoint(): # Save trial runtime if possible diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index f25d0eafef513..7196edca41438 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -36,6 +36,7 @@ def _make_scheduler(args): def _find_checkpoint_dir(exp_list): checkpointable_expts = [exp for exp in exp_list if exp.is_checkpointable()] + logger.info("Searching checkpointable experiments for checkpoint_dir.") if checkpointable_expts: return checkpointable_expts[0].spec["local_dir"] else: @@ -49,6 +50,7 @@ def run_experiments(experiments=None, server_port=TuneServer.DEFAULT_PORT, verbose=True, resume=False, + checkpoint_mode=False, queue_trials=False, trial_executor=None, raise_on_failed_trial=True): @@ -71,6 +73,9 @@ def run_experiments(experiments=None, resume (bool): To resume from full experiment checkpoint. Only the first checkpointable experiment local_dir is checked. If checkpoint exists, the experiment will resume from there. + Requires `checkpoint_mode` to be True (even during resume). + checkpoint_mode: Turns on checkpointing for full Tune experiment. + Currently defaults to False. queue_trials (bool): Whether to queue trials when the cluster does not currently have enough resources to launch one. This should be set to True when running on an autoscaling cluster to enable @@ -99,17 +104,32 @@ def run_experiments(experiments=None, List of Trial objects, holding data for each executed trial. """ - # This is important here to schematize the experiments - # and also does implicit registration. + # This is important to do this here + # because it schematize the experiments + # and it conducts the implicit registration. experiments = convert_to_experiment_list(experiments) checkpoint_dir = _find_checkpoint_dir(experiments) - if resume and checkpoint_dir and os.path.exists( - os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE)): + runner = None + + if resume: + if not checkpoint_mode: + raise ValueError("Resuming Tune experiment run " + "requires `checkpoint_mode`.") + if not checkpoint_dir: + raise ValueError("Did not find a checkpoint_dir. " + "Do any experiments have checkpointing on?") + logger.info("Using checkpoint_dir: {}.".format(checkpoint_dir)) + if not os.path.exists( + os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE)): + raise ValueError( + "Did not find checkpoint file in {}.".format(checkpoint_dir)) + logger.warn("Restoring from previous experiment and " "ignoring any new changes to specification.") runner = TrialRunner.restore(checkpoint_dir, trial_executor) - else: + + if not runner: if scheduler is None: scheduler = FIFOScheduler() @@ -122,6 +142,7 @@ def run_experiments(experiments=None, search_alg, scheduler=scheduler, checkpoint_dir=checkpoint_dir, + checkpoint_mode=checkpoint_mode, launch_web_server=with_server, server_port=server_port, verbose=verbose, From 355d7600f73977ed07c4cfe9d5898bc97a91a2c7 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 19 Dec 2018 23:44:56 -0800 Subject: [PATCH 129/172] docs --- doc/source/tune-usage.rst | 34 ++++++++++++++++++++++++++++++++++ 1 file changed, 34 insertions(+) diff --git a/doc/source/tune-usage.rst b/doc/source/tune-usage.rst index da91e6a4cc364..8f3890ac400d6 100644 --- a/doc/source/tune-usage.rst +++ b/doc/source/tune-usage.rst @@ -296,6 +296,40 @@ of a trial, you can additionally set the checkpoint_at_end to True. An example i }, }) +Recovering From Failures (Experimental) +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +A Tune experiment run may crash due to unforeseen circumstances. To avoid losing all progress, set ``checkpoint_mode=True`` in ``run_experiments``. Note that only **checkpointable experiments** (i.e., experiments with ``checkpoint_freq > 0``) will be preserved. With ``checkpoint_mode=True``, Tune frequently save the entire experiment state at the ``local_dir`` of the first checkpointable experiment provided. E.g.: + +.. code-block:: python + + run_experiments({ + "my_experiment_name": { + "run": my_trainable + "checkpoint_freq": 10, + "local_dir": "~/path/to/results" + }, + }, checkpoint_mode=True) + + +To restore from an experiment, set ``checkpoint_mode=True`` and ``resume=True``, e.g.: + +.. code-block:: python + + run_experiments({ + "my_experiment_name": { + "run": my_trainable + "checkpoint_freq": 10, + "local_dir": "~/path/to/results" + }, + }, checkpoint_mode=True, resume=True) + +This will restore the entire experiment state from the ``local_dir`` of the first checkpointable experiment provided. Importantly, note that any changes to the experiment specification +upon resume will be ignored. + +This feature is still experimental, so any provided Trial Scheduler or Search Algorithm will not be preserved. Only ``FIFOScheduler`` and ``BasicVariantGenerator`` will be supported. + + Handling Large Datasets ----------------------- From f412130718e7e4c6588ac86916a7f3d15bb6fc93 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Thu, 20 Dec 2018 00:59:28 -0800 Subject: [PATCH 130/172] checkpointmode --- python/ray/tune/test/cluster_tests.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 542c0a8f2203d..d14a581061abc 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -274,7 +274,8 @@ def test_cluster_down_simple(start_connected_cluster, tmpdir): assert cluster.wait_for_nodes() dirpath = str(tmpdir) - runner = TrialRunner(BasicVariantGenerator(), checkpoint_dir=dirpath) + runner = TrialRunner( + BasicVariantGenerator(), checkpoint_dir=dirpath, checkpoint_mode=True) kwargs = { "stopping_criterion": { "training_iteration": 2 From 64e4a092a2401ad77cc2697a80d052b493f16820 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 24 Dec 2018 15:05:13 -0800 Subject: [PATCH 131/172] some reversions --- python/ray/tune/test/cluster_tests.py | 13 +++---------- python/ray/tune/tune.py | 25 +++++++++---------------- 2 files changed, 12 insertions(+), 26 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index d14a581061abc..8841e5ba521bd 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -338,15 +338,13 @@ def test_cluster_down_full(start_connected_cluster, tmpdir): "exp4": exp4_args } - tune.run_experiments( - all_experiments, checkpoint_mode=True, raise_on_failed_trial=False) + tune.run_experiments(all_experiments, raise_on_failed_trial=False) ray.shutdown() cluster.shutdown() cluster = _start_new_cluster() - trials = tune.run_experiments( - all_experiments, checkpoint_mode=True, resume=True) + trials = tune.run_experiments(all_experiments, resume=True) assert len(trials) == 2 assert all(t.status in [Trial.TERMINATED, Trial.ERROR] for t in trials) cluster.shutdown() @@ -372,7 +370,6 @@ def test_cluster_rllib_restore(start_connected_cluster, tmpdir): tune.run_experiments( dict(experiment=kwargs), - checkpoint_mode=True, raise_on_failed_trial=False) """.format( redis_address=cluster.redis_address, checkpoint_dir=dirpath) @@ -406,9 +403,7 @@ def test_cluster_rllib_restore(start_connected_cluster, tmpdir): "checkpoint_freq": 1, "local_dir": dirpath } - }, - checkpoint_mode=True, - resume=True) + }, resume=True) assert all(t.status == Trial.TERMINATED for t in trials2) cluster.shutdown() @@ -439,7 +434,6 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): tune.run_experiments( dict(experiment=kwargs), - checkpoint_mode=True, raise_on_failed_trial=False) """.format( redis_address=cluster.redis_address, @@ -485,7 +479,6 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): } }, resume=True, - checkpoint_mode=True, raise_on_failed_trial=False) assert all([t.status == Trial.ERROR for t in trials2]) assert set([t.trial_id for t in trials2]) == set( diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index 7196edca41438..d40d23bface1f 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -50,7 +50,6 @@ def run_experiments(experiments=None, server_port=TuneServer.DEFAULT_PORT, verbose=True, resume=False, - checkpoint_mode=False, queue_trials=False, trial_executor=None, raise_on_failed_trial=True): @@ -70,12 +69,9 @@ def run_experiments(experiments=None, using the Client API. server_port (int): Port number for launching TuneServer. verbose (bool): How much output should be printed for each trial. - resume (bool): To resume from full experiment checkpoint. Only the - first checkpointable experiment local_dir is checked. + resume (bool): Turns on checkpointing. If checkpoint exists, the experiment will resume from there. - Requires `checkpoint_mode` to be True (even during resume). - checkpoint_mode: Turns on checkpointing for full Tune experiment. - Currently defaults to False. + Only the first checkpointable experiment local_dir is checked. queue_trials (bool): Whether to queue trials when the cluster does not currently have enough resources to launch one. This should be set to True when running on an autoscaling cluster to enable @@ -113,21 +109,18 @@ def run_experiments(experiments=None, runner = None if resume: - if not checkpoint_mode: - raise ValueError("Resuming Tune experiment run " - "requires `checkpoint_mode`.") if not checkpoint_dir: raise ValueError("Did not find a checkpoint_dir. " "Do any experiments have checkpointing on?") logger.info("Using checkpoint_dir: {}.".format(checkpoint_dir)) if not os.path.exists( os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE)): - raise ValueError( - "Did not find checkpoint file in {}.".format(checkpoint_dir)) - - logger.warn("Restoring from previous experiment and " - "ignoring any new changes to specification.") - runner = TrialRunner.restore(checkpoint_dir, trial_executor) + logger.warn("Did not find checkpoint file in {}.".format( + checkpoint_dir)) + else: + logger.warn("Restoring from previous experiment and " + "ignoring any new changes to specification.") + runner = TrialRunner.restore(checkpoint_dir, trial_executor) if not runner: if scheduler is None: @@ -142,7 +135,7 @@ def run_experiments(experiments=None, search_alg, scheduler=scheduler, checkpoint_dir=checkpoint_dir, - checkpoint_mode=checkpoint_mode, + checkpoint_mode=resume, launch_web_server=with_server, server_port=server_port, verbose=verbose, From 0e78363f9b972bd6217a4ddb79b8e7972f0452fa Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 24 Dec 2018 17:19:32 -0800 Subject: [PATCH 132/172] Resolve into experiment dir --- python/ray/tune/tune.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index d40d23bface1f..ba16eb818ee3e 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -38,7 +38,9 @@ def _find_checkpoint_dir(exp_list): checkpointable_expts = [exp for exp in exp_list if exp.is_checkpointable()] logger.info("Searching checkpointable experiments for checkpoint_dir.") if checkpointable_expts: - return checkpointable_expts[0].spec["local_dir"] + # TODO(rliaw): This should be resolved in Experiment constructor. + exp = checkpointable_expts[0] + return os.path.join(exp.spec["local_dir"], exp.name) else: return None From 9c9cd9d98477ad7cf3cd6cfcf53825bdec6204ed Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 24 Dec 2018 18:06:56 -0800 Subject: [PATCH 133/172] add example --- ...rband_example.py => checkpoint_example.py} | 19 +++++++------------ 1 file changed, 7 insertions(+), 12 deletions(-) rename python/ray/tune/examples/{hyperband_example.py => checkpoint_example.py} (79%) diff --git a/python/ray/tune/examples/hyperband_example.py b/python/ray/tune/examples/checkpoint_example.py similarity index 79% rename from python/ray/tune/examples/hyperband_example.py rename to python/ray/tune/examples/checkpoint_example.py index d403a0e0f8af1..579217e855081 100755 --- a/python/ray/tune/examples/hyperband_example.py +++ b/python/ray/tune/examples/checkpoint_example.py @@ -8,6 +8,7 @@ import json import os import random +import time import numpy as np @@ -30,7 +31,7 @@ def _train(self): self.timestep += 1 v = np.tanh(float(self.timestep) / self.config["width"]) v *= self.config["height"] - + time.sleep(1) # Here we use `episode_reward_mean`, but you can also report other # objectives such as loss or accuracy. return {"episode_reward_mean": v} @@ -51,24 +52,18 @@ def _restore(self, checkpoint_path): parser.add_argument( "--smoke-test", action="store_true", help="Finish quickly for testing") args, _ = parser.parse_known_args() - ray.init() - - # Hyperband early stopping, configured with `episode_reward_mean` as the - # objective and `training_iteration` as the time unit, - # which is automatically filled by Tune. - hyperband = HyperBandScheduler( - time_attr="training_iteration", - reward_attr="episode_reward_mean", - max_t=100) + ray.init(num_cpus=4) exp = Experiment( name="hyperband_test", run=MyTrainableClass, num_samples=20, - stop={"training_iteration": 1 if args.smoke_test else 99999}, + stop={"training_iteration": 50}, + checkpoint_freq=4, + local_dir="~/ray_results/checkpoint_test/", config={ "width": sample_from(lambda spec: 10 + int(90 * random.random())), "height": sample_from(lambda spec: int(100 * random.random())) }) - run_experiments(exp, scheduler=hyperband) + run_experiments(exp, resume=True) From 3e8641e18ddaca1f83706cf1add13c9c4abe3a41 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 24 Dec 2018 20:47:47 -0800 Subject: [PATCH 134/172] Trial is pickled --- .../ray/tune/examples/checkpoint_example.py | 3 +- python/ray/tune/trial.py | 54 +++++++++++++++---- python/ray/tune/tune.py | 4 +- 3 files changed, 48 insertions(+), 13 deletions(-) diff --git a/python/ray/tune/examples/checkpoint_example.py b/python/ray/tune/examples/checkpoint_example.py index 579217e855081..3ed6c2a5e380f 100755 --- a/python/ray/tune/examples/checkpoint_example.py +++ b/python/ray/tune/examples/checkpoint_example.py @@ -14,7 +14,6 @@ import ray from ray.tune import Trainable, run_experiments, Experiment, sample_from -from ray.tune.schedulers import HyperBandScheduler class MyTrainableClass(Trainable): @@ -60,7 +59,7 @@ def _restore(self, checkpoint_path): num_samples=20, stop={"training_iteration": 50}, checkpoint_freq=4, - local_dir="~/ray_results/checkpoint_test/", + local_dir="~/ray_results/checkpoint_test2/", config={ "width": sample_from(lambda spec: 10 + int(90 * random.random())), "height": sample_from(lambda spec: int(100 * random.random())) diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index 32f02e24bb679..318eb6af6dc39 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -3,6 +3,7 @@ from __future__ import print_function from collections import namedtuple +import cloudpickle from datetime import datetime import logging import time @@ -362,24 +363,59 @@ def __getstate__(self): """ if not self._checkpoint.storage == Checkpoint.DISK: raise ValueError("Checkpoint cannot be in-memory.") - state = self.__dict__.copy() + state = { + "checkpoint_at_end": self.checkpoint_at_end, + "checkpoint_freq": self.checkpoint_freq, + "error_file": self.error_file, + "experiment_tag": self.experiment_tag, + "last_debug": self.last_debug, + "last_result": self.last_result, + "last_update_time": self.last_update_time, + "local_dir": self.local_dir, + "location": self.location, + "logdir": self.logdir, + "max_failures": self.max_failures, + "num_failures": self.num_failures, + "result_logger": None, + "runner": None, + "status": Trial.PENDING + if self.status == Trial.RUNNING else self.status, + "stopping_criterion": self.stopping_criterion, + "trainable_name": self.trainable_name, + "trial_id": self.trial_id, + "upload_dir": self.upload_dir, + "verbose": self.verbose + } + + state.update({ + "_checkpoint": cloudpickle.dumps(self._checkpoint), + "config": cloudpickle.dumps(self.config), + "custom_loggers": cloudpickle.dumps(self.custom_loggers), + "resources": cloudpickle.dumps(self.resources), + "sync_function": cloudpickle.dumps(self.sync_function), + "trial_name_creator": cloudpickle.dumps(self.trial_name_creator), + }) - if state["status"] == Trial.RUNNING: - state["status"] = Trial.PENDING # Remove the unpicklable entries. - if state["result_logger"]: - state["result_logger"].flush() + if self.result_logger: + self.result_logger.flush() state["_logger_started"] = True else: state["_logger_started"] = False - - state["result_logger"] = None - state["runner"] = None return state def __setstate__(self, state): logger_started = state.pop("_logger_started") + state.update({ + "_checkpoint": cloudpickle.loads(state["_checkpoint"]), + "config": cloudpickle.loads(state["config"]), + "custom_loggers": cloudpickle.loads(state["custom_loggers"]), + "resources": cloudpickle.loads(state["resources"]), + "sync_function": cloudpickle.loads(state["sync_function"]), + "trial_name_creator": cloudpickle.loads( + state["trial_name_creator"]), + }) self.__dict__.update(state) - Trial._registration_check(self.__dict__["trainable_name"]) + Trial._registration_check(self.trainable_name) if logger_started: self.init_logger() diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index ba16eb818ee3e..a3c3614265e85 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -117,8 +117,8 @@ def run_experiments(experiments=None, logger.info("Using checkpoint_dir: {}.".format(checkpoint_dir)) if not os.path.exists( os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE)): - logger.warn("Did not find checkpoint file in {}.".format( - checkpoint_dir)) + logger.warn( + "Did not find checkpoint file in {}.".format(checkpoint_dir)) else: logger.warn("Restoring from previous experiment and " "ignoring any new changes to specification.") From c9ca3e2e179509641f8762b41174f9694a59cff2 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 24 Dec 2018 20:59:46 -0800 Subject: [PATCH 135/172] small path fix --- python/ray/tune/experiment.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/python/ray/tune/experiment.py b/python/ray/tune/experiment.py index a9c779c8ef92c..cb6be43a606c4 100644 --- a/python/ray/tune/experiment.py +++ b/python/ray/tune/experiment.py @@ -4,6 +4,7 @@ import copy import logging +import os import six import types @@ -108,7 +109,7 @@ def __init__(self, "config": config or {}, "trial_resources": trial_resources, "num_samples": num_samples, - "local_dir": local_dir or DEFAULT_RESULTS_DIR, + "local_dir": os.path.expanduser(local_dir or DEFAULT_RESULTS_DIR), "upload_dir": upload_dir or "", # argparse converts None to "null" "trial_name_creator": trial_name_creator, "custom_loggers": custom_loggers, From 4e7d438c40e4b53f826e8351b9a7b0c216ecbae5 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 24 Dec 2018 21:33:31 -0800 Subject: [PATCH 136/172] Fix trial serialization --- python/ray/tune/trial.py | 50 +++++++++++++++---------------- python/ray/tune/trial_executor.py | 4 +-- python/ray/tune/trial_runner.py | 8 ++++- 3 files changed, 33 insertions(+), 29 deletions(-) diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index 318eb6af6dc39..d931c8800711d 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -21,7 +21,7 @@ import ray.tune.registry from ray.tune.result import (DEFAULT_RESULTS_DIR, DONE, HOSTNAME, PID, TIME_TOTAL_S, TRAINING_ITERATION, TIMESTEPS_TOTAL) -from ray.utils import random_string, binary_to_hex +from ray.utils import random_string, binary_to_hex, hex_to_binary DEBUG_PRINT_INTERVAL = 5 MAX_LEN_IDENTIFIER = 130 @@ -355,7 +355,7 @@ def __str__(self): identifier += "_" + self.experiment_tag return identifier - def __getstate__(self): + def to_serializable(self): """Memento generator for Trial. Sets RUNNING trials to PENDING, and flushes the result logger. @@ -387,35 +387,33 @@ def __getstate__(self): "verbose": self.verbose } - state.update({ - "_checkpoint": cloudpickle.dumps(self._checkpoint), - "config": cloudpickle.dumps(self.config), - "custom_loggers": cloudpickle.dumps(self.custom_loggers), - "resources": cloudpickle.dumps(self.resources), - "sync_function": cloudpickle.dumps(self.sync_function), - "trial_name_creator": cloudpickle.dumps(self.trial_name_creator), - }) + state["__data__"] = binary_to_hex( + cloudpickle.dumps({ + "_checkpoint": self._checkpoint, + "config": self.config, + "custom_loggers": self.custom_loggers, + "resources": self.resources, + "sync_function": self.sync_function, + "trial_name_creator": self.trial_name_creator, + })) # Remove the unpicklable entries. if self.result_logger: self.result_logger.flush() - state["_logger_started"] = True + state["__logger_started__"] = True else: - state["_logger_started"] = False + state["__logger_started__"] = False return state - def __setstate__(self, state): - logger_started = state.pop("_logger_started") - state.update({ - "_checkpoint": cloudpickle.loads(state["_checkpoint"]), - "config": cloudpickle.loads(state["config"]), - "custom_loggers": cloudpickle.loads(state["custom_loggers"]), - "resources": cloudpickle.loads(state["resources"]), - "sync_function": cloudpickle.loads(state["sync_function"]), - "trial_name_creator": cloudpickle.loads( - state["trial_name_creator"]), - }) - self.__dict__.update(state) - Trial._registration_check(self.trainable_name) + @classmethod + def from_serializable(cls, state): + trial = Trial(state["trainable_name"]) + logger_started = state.pop("__logger_started__") + other_data = cloudpickle.loads(hex_to_binary(state.pop("__data__"))) + state.update(other_data) + + trial.__dict__.update(state) + Trial._registration_check(trial.trainable_name) if logger_started: - self.init_logger() + trial.init_logger() + return trial diff --git a/python/ray/tune/trial_executor.py b/python/ray/tune/trial_executor.py index 246d1ff15f207..eea837316e892 100644 --- a/python/ray/tune/trial_executor.py +++ b/python/ray/tune/trial_executor.py @@ -3,9 +3,9 @@ from __future__ import division from __future__ import print_function +import json import logging -import ray.cloudpickle as cloudpickle from ray.tune.trial import Trial, Checkpoint logger = logging.getLogger(__name__) @@ -58,7 +58,7 @@ def try_checkpoint_metadata(self, trial): return try: logger.debug("Saving trial metadata.") - metadata = cloudpickle.dumps(trial) + metadata = json.dumps(trial.to_serializable()) self._checkpoints[trial.trial_id] = metadata except ValueError: logger.exception("Error checkpointing trial metadata.") diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index a7afd889de448..f30d91637af9b 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -3,6 +3,7 @@ from __future__ import print_function import collections +import json import logging import os import re @@ -162,7 +163,12 @@ def restore(cls, checkpoint_dir, trial_executor=None): RayTrialExecutor(queue_trials=runner._queue_trials) logger.info("Adding all trials with checkpoint state.") - trials = [cloudpickle.loads(cp) for cp in runner_state["checkpoints"]] + trial_checkpoints = [ + json.loads(cp) for cp in runner_state["checkpoints"] + ] + trials = [ + Trial.from_serializable(trial_cp) for trial_cp in trial_checkpoints + ] for trial in sorted( trials, key=lambda t: t.last_update_time, reverse=True): runner.add_trial(trial) From 2318f6ad4050f5966b3b68508107f24a1873dc6e Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Mon, 24 Dec 2018 22:32:42 -0800 Subject: [PATCH 137/172] JSONify state --- python/ray/tune/trial_runner.py | 54 +++++++++++++++------------------ 1 file changed, 24 insertions(+), 30 deletions(-) diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index f30d91637af9b..3250b75956dce 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -98,9 +98,9 @@ def __init__(self, self._queue_trials = queue_trials self._server = None + self._server_port = server_port if launch_web_server: - self._server_port = server_port - self._server = TuneServer(self, server_port) + self._server = TuneServer(self, self._server_port) self._trials = [] self._stop_queue = [] @@ -120,11 +120,11 @@ def checkpoint(self): runner_state = { "checkpoints": list( self.trial_executor.get_checkpoints().values()), - "runner": self + "runner_data": self.__getstate__() } tmp_file_name = os.path.join(checkpoint_dir, ".tmp_checkpoint") - with open(tmp_file_name, "wb") as f: - cloudpickle.dump(runner_state, f) + with open(tmp_file_name, "w") as f: + json.dump(runner_state, f) os.rename(tmp_file_name, os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE)) @@ -146,21 +146,19 @@ def restore(cls, checkpoint_dir, trial_executor=None): runner (TrialRunner): A TrialRunner to resume experiments from. """ with open(os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE), - "rb") as f: - runner_state = cloudpickle.load(f) + "r") as f: + runner_state = json.load(f) logger.warning( "Tune recovery is still experimental. " "There is limited search algorithm recovery support. " "Restoring with a BasicVariantGenerator and FIFOScheduler.") - runner = runner_state["runner"] from ray.tune.suggest import BasicVariantGenerator - runner._search_alg = BasicVariantGenerator() - runner._scheduler_alg = FIFOScheduler() + runner = TrialRunner( + BasicVariantGenerator(), trial_executor=trial_executor) - runner.trial_executor = trial_executor or \ - RayTrialExecutor(queue_trials=runner._queue_trials) + runner.__setstate__(runner_state["runner_data"]) logger.info("Adding all trials with checkpoint state.") trial_checkpoints = [ @@ -507,25 +505,21 @@ def stop_trial(self, trial): self.trial_executor.stop_trial(trial, error=error, error_msg=error_msg) def __getstate__(self): - state = self.__dict__.copy() - - state["trial_executor"] = None - state["_search_alg"] = None - # TODO(rliaw): Remove this once component FT is implemented - state["_scheduler_alg"] = None - - if state["_server"]: - state["_launch_web_server"] = True - state["_server"] = None - - state["_stop_queue"] = [] - state["_trials"] = [] - state["_trial_checkpoints"] = {} + state = { + "_checkpoint_dir": self._checkpoint_dir, + "_checkpoint_mode": self._checkpoint_mode, + "_global_time_limit": self._global_time_limit, + "_iteration": self._iteration, + "_queue_trials": self._queue_trials, + "_server_port": self._server_port, + "_total_time": self._total_time, + "_verbose": self._verbose, + "launch_web_server": bool(self._server) + } return state def __setstate__(self, state): - if "_launch_web_server" in state: - state.pop("_launch_web_server") - state["_server"] = TuneServer(self, state["_server_port"]) - + launch_web_server = state.pop("launch_web_server") self.__dict__.update(state) + if launch_web_server: + self._server = TuneServer(self, self._server_port) From 72d027af135f54c41ea106a12487ce93638f4dc8 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 25 Dec 2018 07:06:09 -0800 Subject: [PATCH 138/172] make sure trials even without checkpointing are not duplicated --- python/ray/tune/trial_executor.py | 7 ++++++- python/ray/tune/trial_runner.py | 3 +-- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/python/ray/tune/trial_executor.py b/python/ray/tune/trial_executor.py index eea837316e892..f525dc42873ad 100644 --- a/python/ray/tune/trial_executor.py +++ b/python/ray/tune/trial_executor.py @@ -49,10 +49,15 @@ def set_status(self, trial, status): def try_checkpoint_metadata(self, trial): """Checkpoints metadata if current session and trial allow. + Metadata checkpointing will occur either if the trial is + checkpointable (meaning its checkpoint frequency is positive), + or if it has not started running + Args: trial (Trial): Trial to checkpoint. """ - if self._checkpoint_mode and trial.checkpoint_freq > 0: + trial_not_started = (trial.status == Trial.PENDING) + if self._checkpoint_mode and (trial.checkpoint_freq > 0 or trial_not_started): if trial._checkpoint.storage == Checkpoint.MEMORY: logger.debug("Not saving data for trial w/ memory checkpoint.") return diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index 3250b75956dce..8b9cd91cfee24 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -104,7 +104,6 @@ def __init__(self, self._trials = [] self._stop_queue = [] - self._trial_checkpoints = {} self._checkpoint_dir = checkpoint_dir self._checkpoint_mode = checkpoint_mode @@ -249,7 +248,7 @@ def add_trial(self, trial): """ trial.set_verbose(self._verbose) self._scheduler_alg.on_trial_add(self, trial) - self._checkpoint_trial_if_needed(trial) + self.trial_executor.try_checkpoint_metadata(trial) self._trials.append(trial) def debug_string(self, max_debug=MAX_DEBUG_TRIALS): From b63f70500b1c7da5c8e710bafef083efbdfaa495 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 25 Dec 2018 07:18:01 -0800 Subject: [PATCH 139/172] small tweaks --- python/ray/tune/ray_trial_executor.py | 6 +++--- python/ray/tune/trial_executor.py | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/python/ray/tune/ray_trial_executor.py b/python/ray/tune/ray_trial_executor.py index 54888e9775fe3..c7ffaacaf6723 100644 --- a/python/ray/tune/ray_trial_executor.py +++ b/python/ray/tune/ray_trial_executor.py @@ -88,6 +88,9 @@ def _stop_trial(self, trial, error=False, error_msg=None, stop_logger (bool): Whether to shut down the trial logger. """ + if stop_logger: + trial.close_logger() + if error: self.set_status(trial, Trial.ERROR) else: @@ -108,9 +111,6 @@ def _stop_trial(self, trial, error=False, error_msg=None, finally: trial.runner = None - if stop_logger: - trial.close_logger() - def start_trial(self, trial, checkpoint=None): """Starts the trial. diff --git a/python/ray/tune/trial_executor.py b/python/ray/tune/trial_executor.py index f525dc42873ad..960c2b26ecf5e 100644 --- a/python/ray/tune/trial_executor.py +++ b/python/ray/tune/trial_executor.py @@ -51,7 +51,7 @@ def try_checkpoint_metadata(self, trial): Metadata checkpointing will occur either if the trial is checkpointable (meaning its checkpoint frequency is positive), - or if it has not started running + or if it has not started running. Args: trial (Trial): Trial to checkpoint. From b764ab73eb5a98e6e8b3aaba1a2ae17d8f23988a Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Tue, 25 Dec 2018 07:29:43 -0800 Subject: [PATCH 140/172] Update doc/source/tune-usage.rst Co-Authored-By: richardliaw --- doc/source/tune-usage.rst | 2 ++ 1 file changed, 2 insertions(+) diff --git a/doc/source/tune-usage.rst b/doc/source/tune-usage.rst index 4f48ab342787c..1d5c052f49b81 100644 --- a/doc/source/tune-usage.rst +++ b/doc/source/tune-usage.rst @@ -298,7 +298,9 @@ of a trial, you can additionally set the checkpoint_at_end to True. An example i Recovering From Failures (Experimental) ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + Tune automatically persists the progress of your experiments, so if an experiment crashes or is otherwise cancelled, it can be resumed by passing `resume=True` in `run_experiments()`. If resume is not enabled, a new experiment will be created instead. You can always force a new experiment to be created by changing the experiment name. +Note that trials will be restored to their last checkpoint. If checkpointing is not enabled, unfinished trials will be restarted from scratch. A Tune experiment run may crash due to unforeseen circumstances. To avoid losing all progress, set ``checkpoint_mode=True`` in ``run_experiments``. Note that only **checkpointable experiments** (i.e., experiments with ``checkpoint_freq > 0``) will be preserved. With ``checkpoint_mode=True``, Tune frequently save the entire experiment state at the ``local_dir`` of the first checkpointable experiment provided. E.g.: .. code-block:: python From 07a36fccdd20f195e20ce0adbc2d7e68575c1141 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Tue, 25 Dec 2018 23:36:37 +0800 Subject: [PATCH 141/172] fix up some components --- doc/source/tune-usage.rst | 15 ++++++++------- python/ray/tune/experiment.py | 1 + python/ray/tune/trial_executor.py | 3 ++- python/ray/tune/trial_runner.py | 16 +++++++++++++--- python/ray/tune/tune.py | 3 ++- 5 files changed, 26 insertions(+), 12 deletions(-) diff --git a/doc/source/tune-usage.rst b/doc/source/tune-usage.rst index 1d5c052f49b81..0291ada805cd8 100644 --- a/doc/source/tune-usage.rst +++ b/doc/source/tune-usage.rst @@ -298,10 +298,12 @@ of a trial, you can additionally set the checkpoint_at_end to True. An example i Recovering From Failures (Experimental) ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - Tune automatically persists the progress of your experiments, so if an experiment crashes or is otherwise cancelled, it can be resumed by passing `resume=True` in `run_experiments()`. If resume is not enabled, a new experiment will be created instead. You can always force a new experiment to be created by changing the experiment name. + +Tune automatically persists the progress of your experiments, so if an experiment crashes or is otherwise cancelled, it can be resumed by passing `resume=True` in `run_experiments()`. If resume is not enabled, a new experiment will be created instead. You can always force a new experiment to be created by changing the experiment name. Note that trials will be restored to their last checkpoint. If checkpointing is not enabled, unfinished trials will be restarted from scratch. -A Tune experiment run may crash due to unforeseen circumstances. To avoid losing all progress, set ``checkpoint_mode=True`` in ``run_experiments``. Note that only **checkpointable experiments** (i.e., experiments with ``checkpoint_freq > 0``) will be preserved. With ``checkpoint_mode=True``, Tune frequently save the entire experiment state at the ``local_dir`` of the first checkpointable experiment provided. E.g.: + +E.g.: .. code-block:: python @@ -311,10 +313,10 @@ A Tune experiment run may crash due to unforeseen circumstances. To avoid losing "checkpoint_freq": 10, "local_dir": "~/path/to/results" }, - }, checkpoint_mode=True) + }, resume=True) -To restore from an experiment, set ``checkpoint_mode=True`` and ``resume=True``, e.g.: +To restore from an experiment, e.g.: .. code-block:: python @@ -324,10 +326,9 @@ To restore from an experiment, set ``checkpoint_mode=True`` and ``resume=True``, "checkpoint_freq": 10, "local_dir": "~/path/to/results" }, - }, checkpoint_mode=True, resume=True) + }, resume=True) -This will restore the entire experiment state from the ``local_dir`` of the first checkpointable experiment provided. Importantly, note that any changes to the experiment specification -upon resume will be ignored. +This will restore the entire experiment state from the ``local_dir/my_experiment_name`` of the first checkpointable experiment provided. Importantly, note that any changes to the experiment specification upon resume will be ignored. This feature is still experimental, so any provided Trial Scheduler or Search Algorithm will not be preserved. Only ``FIFOScheduler`` and ``BasicVariantGenerator`` will be supported. diff --git a/python/ray/tune/experiment.py b/python/ray/tune/experiment.py index 3ed3a933fda5d..00c2d6f530e17 100644 --- a/python/ray/tune/experiment.py +++ b/python/ray/tune/experiment.py @@ -9,6 +9,7 @@ import types from ray.tune.error import TuneError +from ray.tune.log_sync import validate_sync_function from ray.tune.registry import register_trainable from ray.tune.result import DEFAULT_RESULTS_DIR diff --git a/python/ray/tune/trial_executor.py b/python/ray/tune/trial_executor.py index 960c2b26ecf5e..8502dc7f9b0b1 100644 --- a/python/ray/tune/trial_executor.py +++ b/python/ray/tune/trial_executor.py @@ -57,7 +57,8 @@ def try_checkpoint_metadata(self, trial): trial (Trial): Trial to checkpoint. """ trial_not_started = (trial.status == Trial.PENDING) - if self._checkpoint_mode and (trial.checkpoint_freq > 0 or trial_not_started): + if self._checkpoint_mode and (trial.checkpoint_freq > 0 + or trial_not_started): if trial._checkpoint.storage == Checkpoint.MEMORY: logger.debug("Not saving data for trial w/ memory checkpoint.") return diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index 8b9cd91cfee24..c3b08e70d853a 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -10,7 +10,6 @@ import time import traceback -import ray.cloudpickle as cloudpickle from ray.tune import TuneError from ray.tune.ray_trial_executor import RayTrialExecutor from ray.tune.result import TIME_THIS_ITER_S @@ -130,7 +129,11 @@ def checkpoint(self): return checkpoint_dir @classmethod - def restore(cls, checkpoint_dir, trial_executor=None): + def restore(cls, + checkpoint_dir, + search_alg=None, + scheduler=None, + trial_executor=None): """Restores all checkpointed trials from previous run. Requires user to manually re-register their objects. Also stops @@ -140,6 +143,11 @@ def restore(cls, checkpoint_dir, trial_executor=None): Args: checkpoint_dir (str): Path to checkpoint (previously specified). + search_alg (SearchAlgorithm): Search Algorithm. Defaults to + BasicVariantGenerator. + scheduler (TrialScheduler): Scheduler for executing + the experiment. + trial_executor (TrialExecutor): Manage the execution of trials. Returns: runner (TrialRunner): A TrialRunner to resume experiments from. @@ -155,7 +163,9 @@ def restore(cls, checkpoint_dir, trial_executor=None): from ray.tune.suggest import BasicVariantGenerator runner = TrialRunner( - BasicVariantGenerator(), trial_executor=trial_executor) + search_alg or BasicVariantGenerator(), + scheduler=scheduler, + trial_executor=trial_executor) runner.__setstate__(runner_state["runner_data"]) diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index a3c3614265e85..1d6fb2acf546e 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -122,7 +122,8 @@ def run_experiments(experiments=None, else: logger.warn("Restoring from previous experiment and " "ignoring any new changes to specification.") - runner = TrialRunner.restore(checkpoint_dir, trial_executor) + runner = TrialRunner.restore(checkpoint_dir, search_alg, scheduler, + trial_executor) if not runner: if scheduler is None: From c2eddefa2105308526734f6c5864f5ec2ac0731e Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 26 Dec 2018 10:02:39 +0800 Subject: [PATCH 142/172] remove accidental merge --- python/ray/tune/experiment.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/python/ray/tune/experiment.py b/python/ray/tune/experiment.py index 00c2d6f530e17..51f45f07e650d 100644 --- a/python/ray/tune/experiment.py +++ b/python/ray/tune/experiment.py @@ -9,7 +9,6 @@ import types from ray.tune.error import TuneError -from ray.tune.log_sync import validate_sync_function from ray.tune.registry import register_trainable from ray.tune.result import DEFAULT_RESULTS_DIR @@ -122,7 +121,6 @@ def __init__(self, restore=None, repeat=None, trial_resources=None): - validate_sync_function(sync_function) if sync_function: assert upload_dir, "Need `upload_dir` if sync_function given." From 27fecb1af836d69e8f514a36dd16ffba5ba664ab Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 26 Dec 2018 11:17:16 +0800 Subject: [PATCH 143/172] Fix test for uncheckpointables --- python/ray/tune/test/trial_runner_test.py | 60 ++++++++++++++++------- python/ray/tune/trial_executor.py | 10 +--- python/ray/tune/trial_runner.py | 3 +- 3 files changed, 45 insertions(+), 28 deletions(-) diff --git a/python/ray/tune/test/trial_runner_test.py b/python/ray/tune/test/trial_runner_test.py index 6fcd570810d91..977eb8cc653e9 100644 --- a/python/ray/tune/test/trial_runner_test.py +++ b/python/ray/tune/test/trial_runner_test.py @@ -1656,16 +1656,17 @@ def testTrialSaveRestore(self): """Creates different trials to test runner.checkpoint/restore.""" ray.init(num_cpus=3) tmpdir = tempfile.mkdtemp() - default_resources = Resources(cpu=1, gpu=0) - runner = TrialRunner(BasicVariantGenerator(), checkpoint_dir=tmpdir) + runner = TrialRunner( + BasicVariantGenerator(), + checkpoint_mode=True, + checkpoint_dir=tmpdir) trials = [ Trial( "__fake", trial_id="trial_terminate", stopping_criterion={"training_iteration": 1}, - checkpoint_freq=1, - resources=default_resources) + checkpoint_freq=1) ] runner.add_trial(trials[0]) runner.step() # start @@ -1678,8 +1679,7 @@ def testTrialSaveRestore(self): trial_id="trial_fail", stopping_criterion={"training_iteration": 3}, checkpoint_freq=1, - config={"mock_error": True}, - resources=default_resources) + config={"mock_error": True}) ] runner.add_trial(trials[1]) runner.step() @@ -1692,8 +1692,7 @@ def testTrialSaveRestore(self): "__fake", trial_id="trial_succ", stopping_criterion={"training_iteration": 2}, - checkpoint_freq=1, - resources=default_resources) + checkpoint_freq=1) ] runner.add_trial(trials[2]) runner.step() @@ -1719,24 +1718,47 @@ def testTrialNoSave(self): """Check that non-checkpointing 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) - trials = [ + runner = TrialRunner( + BasicVariantGenerator(), + checkpoint_mode=True, + checkpoint_dir=tmpdir) + + runner.add_trial( Trial( "__fake", - trial_id="trial_terminate", - stopping_criterion={"training_iteration": 2}, - resources=default_resources) - ] - runner.add_trial(trials[0]) - runner.step() # start + trial_id="non_checkpoint", + stopping_criterion={"training_iteration": 2})) + + while not all(t.status == Trial.TERMINATED for t in runner.get_trials()): + runner.step() + + runner.add_trial( + Trial( + "__fake", + trial_id="checkpoint", + checkpoint_at_end=True, + stopping_criterion={"training_iteration": 2})) + + while not all(t.status == Trial.TERMINATED for t in runner.get_trials()): + runner.step() + + runner.add_trial( + Trial( + "__fake", + trial_id="pending", + stopping_criterion={"training_iteration": 2})) + + runner.step() runner.step() runner2 = TrialRunner.restore(tmpdir) - self.assertEquals(len(runner2.get_trials()), 0) + new_trials = runner2.get_trials() + self.assertEquals(len(new_trials), 3) + self.assertTrue(runner2.get_trial("non_checkpoint").status == Trial.TERMINATED) + self.assertTrue(runner2.get_trial("checkpoint").status == Trial.TERMINATED) + self.assertTrue(runner2.get_trial("pending").status == Trial.PENDING) runner2.step() - self.assertRaises(TuneError, runner2.step) shutil.rmtree(tmpdir) diff --git a/python/ray/tune/trial_executor.py b/python/ray/tune/trial_executor.py index 8502dc7f9b0b1..8be47a24e8d7a 100644 --- a/python/ray/tune/trial_executor.py +++ b/python/ray/tune/trial_executor.py @@ -47,18 +47,12 @@ def set_status(self, trial, status): self.try_checkpoint_metadata(trial) def try_checkpoint_metadata(self, trial): - """Checkpoints metadata if current session and trial allow. - - Metadata checkpointing will occur either if the trial is - checkpointable (meaning its checkpoint frequency is positive), - or if it has not started running. + """Checkpoints metadata if checkpoint_mode is True. Args: trial (Trial): Trial to checkpoint. """ - trial_not_started = (trial.status == Trial.PENDING) - if self._checkpoint_mode and (trial.checkpoint_freq > 0 - or trial_not_started): + if self._checkpoint_mode: if trial._checkpoint.storage == Checkpoint.MEMORY: logger.debug("Not saving data for trial w/ memory checkpoint.") return diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index c3b08e70d853a..81a3089bad698 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -258,7 +258,8 @@ def add_trial(self, trial): """ trial.set_verbose(self._verbose) self._scheduler_alg.on_trial_add(self, trial) - self.trial_executor.try_checkpoint_metadata(trial) + if trial.status == Trial.PENDING: + self.trial_executor.try_checkpoint_metadata(trial) self._trials.append(trial) def debug_string(self, max_debug=MAX_DEBUG_TRIALS): From bc8e67a88d9ae4dc8e96c17cf051fb37106bac09 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 26 Dec 2018 11:23:15 +0800 Subject: [PATCH 144/172] back to dict --- python/ray/tune/trial.py | 27 ++++----------------------- python/ray/tune/trial_executor.py | 2 +- 2 files changed, 5 insertions(+), 24 deletions(-) diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index 9c8f8bdb6a6a4..e0b17b0e56636 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -363,29 +363,7 @@ def to_serializable(self): """ if not self._checkpoint.storage == Checkpoint.DISK: raise ValueError("Checkpoint cannot be in-memory.") - state = { - "checkpoint_at_end": self.checkpoint_at_end, - "checkpoint_freq": self.checkpoint_freq, - "error_file": self.error_file, - "experiment_tag": self.experiment_tag, - "last_debug": self.last_debug, - "last_result": self.last_result, - "last_update_time": self.last_update_time, - "local_dir": self.local_dir, - "location": self.location, - "logdir": self.logdir, - "max_failures": self.max_failures, - "num_failures": self.num_failures, - "result_logger": None, - "runner": None, - "status": Trial.PENDING - if self.status == Trial.RUNNING else self.status, - "stopping_criterion": self.stopping_criterion, - "trainable_name": self.trainable_name, - "trial_id": self.trial_id, - "upload_dir": self.upload_dir, - "verbose": self.verbose - } + state = copy.deepcopy(self.__dict__) state["__data__"] = binary_to_hex( cloudpickle.dumps({ @@ -398,6 +376,9 @@ def to_serializable(self): })) # Remove the unpicklable entries. + state["runner"] = None + state["result_logger"] = None + state["status"] = Trial.PENDING if self.status == Trial.RUNNING else self.status if self.result_logger: self.result_logger.flush() state["__logger_started__"] = True diff --git a/python/ray/tune/trial_executor.py b/python/ray/tune/trial_executor.py index 8be47a24e8d7a..d4cdcc6ed3a8e 100644 --- a/python/ray/tune/trial_executor.py +++ b/python/ray/tune/trial_executor.py @@ -60,7 +60,7 @@ def try_checkpoint_metadata(self, trial): logger.debug("Saving trial metadata.") metadata = json.dumps(trial.to_serializable()) self._checkpoints[trial.trial_id] = metadata - except ValueError: + except Exception: logger.exception("Error checkpointing trial metadata.") def get_checkpoints(self): From de1c14bae9548eb0d0605baf38b67de145111376 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 26 Dec 2018 14:43:25 +0800 Subject: [PATCH 145/172] __getstate__, remove checkpointmode, trial name --- python/ray/tune/test/cluster_tests.py | 6 ++-- python/ray/tune/test/trial_runner_test.py | 22 ++++++------ python/ray/tune/trial.py | 26 +++++++------- python/ray/tune/trial_executor.py | 2 +- python/ray/tune/trial_runner.py | 41 ++++++++++++----------- 5 files changed, 48 insertions(+), 49 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 8841e5ba521bd..6bed64940acf2 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -274,8 +274,7 @@ def test_cluster_down_simple(start_connected_cluster, tmpdir): assert cluster.wait_for_nodes() dirpath = str(tmpdir) - runner = TrialRunner( - BasicVariantGenerator(), checkpoint_dir=dirpath, checkpoint_mode=True) + runner = TrialRunner(BasicVariantGenerator(), checkpoint_dir=dirpath) kwargs = { "stopping_criterion": { "training_iteration": 2 @@ -403,7 +402,8 @@ def test_cluster_rllib_restore(start_connected_cluster, tmpdir): "checkpoint_freq": 1, "local_dir": dirpath } - }, resume=True) + }, + resume=True) assert all(t.status == Trial.TERMINATED for t in trials2) cluster.shutdown() diff --git a/python/ray/tune/test/trial_runner_test.py b/python/ray/tune/test/trial_runner_test.py index 977eb8cc653e9..cd2d3d45dcee6 100644 --- a/python/ray/tune/test/trial_runner_test.py +++ b/python/ray/tune/test/trial_runner_test.py @@ -1657,10 +1657,7 @@ def testTrialSaveRestore(self): ray.init(num_cpus=3) tmpdir = tempfile.mkdtemp() - runner = TrialRunner( - BasicVariantGenerator(), - checkpoint_mode=True, - checkpoint_dir=tmpdir) + runner = TrialRunner(BasicVariantGenerator(), checkpoint_dir=tmpdir) trials = [ Trial( "__fake", @@ -1719,10 +1716,7 @@ def testTrialNoSave(self): ray.init(num_cpus=3) tmpdir = tempfile.mkdtemp() - runner = TrialRunner( - BasicVariantGenerator(), - checkpoint_mode=True, - checkpoint_dir=tmpdir) + runner = TrialRunner(BasicVariantGenerator(), checkpoint_dir=tmpdir) runner.add_trial( Trial( @@ -1730,7 +1724,8 @@ def testTrialNoSave(self): trial_id="non_checkpoint", stopping_criterion={"training_iteration": 2})) - while not all(t.status == Trial.TERMINATED for t in runner.get_trials()): + while not all(t.status == Trial.TERMINATED + for t in runner.get_trials()): runner.step() runner.add_trial( @@ -1740,7 +1735,8 @@ def testTrialNoSave(self): checkpoint_at_end=True, stopping_criterion={"training_iteration": 2})) - while not all(t.status == Trial.TERMINATED for t in runner.get_trials()): + while not all(t.status == Trial.TERMINATED + for t in runner.get_trials()): runner.step() runner.add_trial( @@ -1755,8 +1751,10 @@ def testTrialNoSave(self): runner2 = TrialRunner.restore(tmpdir) new_trials = runner2.get_trials() self.assertEquals(len(new_trials), 3) - self.assertTrue(runner2.get_trial("non_checkpoint").status == Trial.TERMINATED) - self.assertTrue(runner2.get_trial("checkpoint").status == Trial.TERMINATED) + self.assertTrue( + runner2.get_trial("non_checkpoint").status == Trial.TERMINATED) + self.assertTrue( + runner2.get_trial("checkpoint").status == Trial.TERMINATED) self.assertTrue(runner2.get_trial("pending").status == Trial.PENDING) runner2.step() shutil.rmtree(tmpdir) diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index e0b17b0e56636..4539bb219f53f 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -147,7 +147,6 @@ def __init__(self, or self._get_trainable_cls().default_resource_request(self.config)) self.stopping_criterion = stopping_criterion or {} self.upload_dir = upload_dir - self.trial_name_creator = trial_name_creator self.custom_loggers = custom_loggers self.sync_function = sync_function validate_sync_function(sync_function) @@ -170,6 +169,11 @@ def __init__(self, self.error_file = None self.num_failures = 0 + self.trial_name = None + if trial_name_creator: + self.trial_name = trial_name_creator(self) + + @classmethod def _registration_check(cls, trainable_name): if not has_trainable(trainable_name): @@ -341,8 +345,8 @@ def __str__(self): Can be overriden with a custom string creator. """ - if self.trial_name_creator: - return self.trial_name_creator(self) + if self.trial_name: + return self.trial_name if "env" in self.config: env = self.config["env"] @@ -355,7 +359,7 @@ def __str__(self): identifier += "_" + self.experiment_tag return identifier.replace("/", "_") - def to_serializable(self): + def __getstate__(self): """Memento generator for Trial. Sets RUNNING trials to PENDING, and flushes the result logger. @@ -371,8 +375,7 @@ def to_serializable(self): "config": self.config, "custom_loggers": self.custom_loggers, "resources": self.resources, - "sync_function": self.sync_function, - "trial_name_creator": self.trial_name_creator, + "sync_function": self.sync_function })) # Remove the unpicklable entries. @@ -386,15 +389,12 @@ def to_serializable(self): state["__logger_started__"] = False return state - @classmethod - def from_serializable(cls, state): - trial = Trial(state["trainable_name"]) + def __setstate__(self, state): logger_started = state.pop("__logger_started__") other_data = cloudpickle.loads(hex_to_binary(state.pop("__data__"))) state.update(other_data) - trial.__dict__.update(state) - Trial._registration_check(trial.trainable_name) + self.__dict__.update(state) + Trial._registration_check(self.trainable_name) if logger_started: - trial.init_logger() - return trial + self.init_logger() diff --git a/python/ray/tune/trial_executor.py b/python/ray/tune/trial_executor.py index d4cdcc6ed3a8e..74f531985b2d1 100644 --- a/python/ray/tune/trial_executor.py +++ b/python/ray/tune/trial_executor.py @@ -58,7 +58,7 @@ def try_checkpoint_metadata(self, trial): return try: logger.debug("Saving trial metadata.") - metadata = json.dumps(trial.to_serializable()) + metadata = json.dumps(trial.__getstate__(), indent=2) self._checkpoints[trial.trial_id] = metadata except Exception: logger.exception("Error checkpointing trial metadata.") diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index 81a3089bad698..f01db548aafd8 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -57,7 +57,6 @@ def __init__(self, scheduler=None, launch_web_server=False, checkpoint_dir=None, - checkpoint_mode=False, server_port=TuneServer.DEFAULT_PORT, verbose=True, queue_trials=False, @@ -71,8 +70,6 @@ def __init__(self, launch_web_server (bool): Flag for starting TuneServer checkpoint_dir (str): Path where global checkpoints are stored and restored from. - checkpoint_mode: Turns on checkpointing for full Tune experiment. - Currently defaults to False. server_port (int): Port number for launching TuneServer verbose (bool): Flag for verbosity. If False, trial results will not be output. @@ -104,11 +101,10 @@ def __init__(self, self._trials = [] self._stop_queue = [] self._checkpoint_dir = checkpoint_dir - self._checkpoint_mode = checkpoint_mode def checkpoint(self): """Saves execution state to `self._checkpoint_dir` if provided.""" - if not self._checkpoint_mode or not self._checkpoint_dir: + if not self._checkpoint_dir: return checkpoint_dir = self._checkpoint_dir if not os.path.exists(checkpoint_dir): @@ -173,9 +169,11 @@ def restore(cls, trial_checkpoints = [ json.loads(cp) for cp in runner_state["checkpoints"] ] - trials = [ - Trial.from_serializable(trial_cp) for trial_cp in trial_checkpoints - ] + trials = [] + for trial_cp in trial_checkpoints: + new_trial = Trial(trial_cp["trainable_name"]) + new_trial.__setstate__(trial_cp) + trials += [new_trial] for trial in sorted( trials, key=lambda t: t.last_update_time, reverse=True): runner.add_trial(trial) @@ -226,7 +224,10 @@ def step(self): "There are paused trials, but no more pending " "trials with sufficient resources.") - self.checkpoint() + try: + self.checkpoint() + except Exception: + logger.exception("Trial Runner checkpointing failed.") self._iteration += 1 if self._server: @@ -515,17 +516,17 @@ def stop_trial(self, trial): self.trial_executor.stop_trial(trial, error=error, error_msg=error_msg) def __getstate__(self): - state = { - "_checkpoint_dir": self._checkpoint_dir, - "_checkpoint_mode": self._checkpoint_mode, - "_global_time_limit": self._global_time_limit, - "_iteration": self._iteration, - "_queue_trials": self._queue_trials, - "_server_port": self._server_port, - "_total_time": self._total_time, - "_verbose": self._verbose, - "launch_web_server": bool(self._server) - } + """Gets state for trial. + + Note that this is not used as a pickling override as + does not have all fields. + """ + state = copy.deepcopy(self.__dict__) + for k in [ + "trials", "_stop_queue", "_server", "_search_alg", + "_scheduler_alg", "trial_executor", "launch_web_server" + ]: + del state[k] return state def __setstate__(self, state): From 33dac8f3f54ab8a424b1e293cea56b33ad4cb578 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 26 Dec 2018 14:52:11 +0800 Subject: [PATCH 146/172] resources_to_json --- python/ray/rllib/train.py | 3 ++- python/ray/tune/config_parser.py | 32 +------------------------ python/ray/tune/trial.py | 41 +++++++++++++++++++++++++++++--- python/ray/tune/trial_runner.py | 1 + 4 files changed, 42 insertions(+), 35 deletions(-) diff --git a/python/ray/rllib/train.py b/python/ray/rllib/train.py index d9f7cf58e0b4c..39a06d0d02b0c 100755 --- a/python/ray/rllib/train.py +++ b/python/ray/rllib/train.py @@ -9,7 +9,8 @@ import ray from ray.test.cluster_utils import Cluster -from ray.tune.config_parser import make_parser, resources_to_json +from ray.tune.config_parser import make_parser +from ray.tune.trial import resources_to_json from ray.tune.tune import _make_scheduler, run_experiments EXAMPLE_USAGE = """ diff --git a/python/ray/tune/config_parser.py b/python/ray/tune/config_parser.py index 22adfc397ecc6..aa0caa437ae68 100644 --- a/python/ray/tune/config_parser.py +++ b/python/ray/tune/config_parser.py @@ -11,40 +11,10 @@ from ray.tune import TuneError from ray.tune.result import DEFAULT_RESULTS_DIR -from ray.tune.trial import Resources, Trial +from ray.tune.trial import Trial, json_to_resources from ray.tune.logger import _SafeFallbackEncoder -def json_to_resources(data): - if data is None or data == "null": - return None - if isinstance(data, string_types): - data = json.loads(data) - for k in data: - if k in ["driver_cpu_limit", "driver_gpu_limit"]: - raise TuneError( - "The field `{}` is no longer supported. Use `extra_cpu` " - "or `extra_gpu` instead.".format(k)) - if k not in Resources._fields: - raise TuneError( - "Unknown resource type {}, must be one of {}".format( - k, Resources._fields)) - return Resources( - data.get("cpu", 1), data.get("gpu", 0), data.get("extra_cpu", 0), - data.get("extra_gpu", 0)) - - -def resources_to_json(resources): - if resources is None: - return None - return { - "cpu": resources.cpu, - "gpu": resources.gpu, - "extra_cpu": resources.extra_cpu, - "extra_gpu": resources.extra_gpu, - } - - def make_parser(parser_creator=None, **kwargs): """Returns a base argument parser for the ray.tune tool. diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index 4539bb219f53f..d7bb12d7ed92d 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -4,11 +4,16 @@ from collections import namedtuple import cloudpickle +import copy from datetime import datetime import logging +import json import time import tempfile import os + +# For compatibility under py2 to consider unicode as str +from six import string_types from numbers import Number import ray @@ -68,6 +73,36 @@ def gpu_total(self): return self.gpu + self.extra_gpu +def json_to_resources(data): + if data is None or data == "null": + return None + if isinstance(data, string_types): + data = json.loads(data) + for k in data: + if k in ["driver_cpu_limit", "driver_gpu_limit"]: + raise TuneError( + "The field `{}` is no longer supported. Use `extra_cpu` " + "or `extra_gpu` instead.".format(k)) + if k not in Resources._fields: + raise TuneError( + "Unknown resource type {}, must be one of {}".format( + k, Resources._fields)) + return Resources( + data.get("cpu", 1), data.get("gpu", 0), data.get("extra_cpu", 0), + data.get("extra_gpu", 0)) + + +def resources_to_json(resources): + if resources is None: + return None + return { + "cpu": resources.cpu, + "gpu": resources.gpu, + "extra_cpu": resources.extra_cpu, + "extra_gpu": resources.extra_gpu, + } + + def has_trainable(trainable_name): return ray.tune.registry._global_registry.contains( ray.tune.registry.TRAINABLE_CLASS, trainable_name) @@ -173,7 +208,6 @@ def __init__(self, if trial_name_creator: self.trial_name = trial_name_creator(self) - @classmethod def _registration_check(cls, trainable_name): if not has_trainable(trainable_name): @@ -374,14 +408,15 @@ def __getstate__(self): "_checkpoint": self._checkpoint, "config": self.config, "custom_loggers": self.custom_loggers, - "resources": self.resources, + "resources": resources_to_json(self.resources), "sync_function": self.sync_function })) # Remove the unpicklable entries. state["runner"] = None state["result_logger"] = None - state["status"] = Trial.PENDING if self.status == Trial.RUNNING else self.status + if self.status == Trial.RUNNING: + state["status"] = Trial.PENDING if self.result_logger: self.result_logger.flush() state["__logger_started__"] = True diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index f01db548aafd8..a9e5a6dac97c7 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -3,6 +3,7 @@ from __future__ import print_function import collections +import copy import json import logging import os From 763e6d4d05bf4a7676283eea2908295c19d1d18b Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 26 Dec 2018 15:21:14 +0800 Subject: [PATCH 147/172] Fix experiments and final comments. --- python/ray/tune/experiment.py | 5 ----- python/ray/tune/trial.py | 23 +++++++++++++---------- python/ray/tune/trial_runner.py | 10 +++++----- python/ray/tune/tune.py | 24 +++++++++--------------- 4 files changed, 27 insertions(+), 35 deletions(-) diff --git a/python/ray/tune/experiment.py b/python/ray/tune/experiment.py index 51f45f07e650d..7e08899b91440 100644 --- a/python/ray/tune/experiment.py +++ b/python/ray/tune/experiment.py @@ -151,11 +151,6 @@ def __init__(self, self.name = name self.spec = spec - def is_checkpointable(self): - """Returns whether any trial in experiment will need to checkpoint.""" - return bool(self.spec["checkpoint_freq"] - or self.spec["checkpoint_at_end"]) - @classmethod def from_json(cls, name, spec): """Generates an Experiment object from JSON. diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index d7bb12d7ed92d..aa4e3c0cda38b 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -4,7 +4,6 @@ from collections import namedtuple import cloudpickle -import copy from datetime import datetime import logging import json @@ -401,18 +400,21 @@ def __getstate__(self): """ if not self._checkpoint.storage == Checkpoint.DISK: raise ValueError("Checkpoint cannot be in-memory.") - state = copy.deepcopy(self.__dict__) + state = self.__dict__.copy() - state["__data__"] = binary_to_hex( - cloudpickle.dumps({ - "_checkpoint": self._checkpoint, - "config": self.config, - "custom_loggers": self.custom_loggers, - "resources": resources_to_json(self.resources), - "sync_function": self.sync_function - })) + pickle_data = { + "_checkpoint": self._checkpoint, + "config": self.config, + "custom_loggers": self.custom_loggers, + "resources": resources_to_json(self.resources), + "sync_function": self.sync_function + } + state["__data__"] = binary_to_hex(cloudpickle.dumps(pickle_data)) # Remove the unpicklable entries. + for key in pickle_data: + state[key] = None + state["runner"] = None state["result_logger"] = None if self.status == Trial.RUNNING: @@ -427,6 +429,7 @@ def __getstate__(self): def __setstate__(self, state): logger_started = state.pop("__logger_started__") other_data = cloudpickle.loads(hex_to_binary(state.pop("__data__"))) + other_data["resources"] = json_to_resources(other_data["resources"]) state.update(other_data) self.__dict__.update(state) diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index a9e5a6dac97c7..8b46bd6843541 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -3,7 +3,6 @@ from __future__ import print_function import collections -import copy import json import logging import os @@ -166,7 +165,7 @@ def restore(cls, runner.__setstate__(runner_state["runner_data"]) - logger.info("Adding all trials with checkpoint state.") + logger.info("Adding trials.") trial_checkpoints = [ json.loads(cp) for cp in runner_state["checkpoints"] ] @@ -522,12 +521,13 @@ def __getstate__(self): Note that this is not used as a pickling override as does not have all fields. """ - state = copy.deepcopy(self.__dict__) + state = self.__dict__.copy() for k in [ - "trials", "_stop_queue", "_server", "_search_alg", - "_scheduler_alg", "trial_executor", "launch_web_server" + "_trials", "_stop_queue", "_server", "_search_alg", + "_scheduler_alg", "trial_executor" ]: del state[k] + state["launch_web_server"] = bool(self._server) return state def __setstate__(self, state): diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index 1d6fb2acf546e..7990df0ab05f9 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -35,14 +35,9 @@ def _make_scheduler(args): def _find_checkpoint_dir(exp_list): - checkpointable_expts = [exp for exp in exp_list if exp.is_checkpointable()] - logger.info("Searching checkpointable experiments for checkpoint_dir.") - if checkpointable_expts: - # TODO(rliaw): This should be resolved in Experiment constructor. - exp = checkpointable_expts[0] - return os.path.join(exp.spec["local_dir"], exp.name) - else: - return None + exp = exp_list[0] + # TODO(rliaw): Make sure this is resolved earlier. + return os.path.join(exp.spec["local_dir"], exp.name) def run_experiments(experiments=None, @@ -111,10 +106,7 @@ def run_experiments(experiments=None, runner = None if resume: - if not checkpoint_dir: - raise ValueError("Did not find a checkpoint_dir. " - "Do any experiments have checkpointing on?") - logger.info("Using checkpoint_dir: {}.".format(checkpoint_dir)) + logger.info("Using checkpoint dir: {}.".format(checkpoint_dir)) if not os.path.exists( os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE)): logger.warn( @@ -122,8 +114,11 @@ def run_experiments(experiments=None, else: logger.warn("Restoring from previous experiment and " "ignoring any new changes to specification.") - runner = TrialRunner.restore(checkpoint_dir, search_alg, scheduler, - trial_executor) + try: + runner = TrialRunner.restore(checkpoint_dir, search_alg, + scheduler, trial_executor) + except Exception: + logger.info("Runner restore failed. Restarting experiment.") if not runner: if scheduler is None: @@ -138,7 +133,6 @@ def run_experiments(experiments=None, search_alg, scheduler=scheduler, checkpoint_dir=checkpoint_dir, - checkpoint_mode=resume, launch_web_server=with_server, server_port=server_port, verbose=verbose, From 1a1523747649749ee40cc6fb20101f74a756048c Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 26 Dec 2018 16:43:33 +0800 Subject: [PATCH 148/172] Fix more changes --- python/ray/tune/ray_trial_executor.py | 2 ++ python/ray/tune/trial.py | 6 +++--- python/ray/tune/trial_executor.py | 3 +-- python/ray/tune/trial_runner.py | 16 +++++----------- python/ray/tune/tune.py | 3 ++- 5 files changed, 13 insertions(+), 17 deletions(-) diff --git a/python/ray/tune/ray_trial_executor.py b/python/ray/tune/ray_trial_executor.py index c7ffaacaf6723..5e0a7f5660c2a 100644 --- a/python/ray/tune/ray_trial_executor.py +++ b/python/ray/tune/ray_trial_executor.py @@ -39,6 +39,8 @@ def _setup_runner(self, trial): num_gpus=trial.resources.gpu)(trial._get_trainable_cls()) trial.init_logger() + # We checkpoint metadata here to try mitigating logdir duplication + self.try_checkpoint_metadata(trial) remote_logdir = trial.logdir def logger_creator(config): diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index aa4e3c0cda38b..c2f5e72e4c3bd 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -4,6 +4,7 @@ from collections import namedtuple import cloudpickle +import copy from datetime import datetime import logging import json @@ -225,14 +226,13 @@ def init_logger(self): if not self.result_logger: if not os.path.exists(self.local_dir): os.makedirs(self.local_dir) - if not self.logdir: self.logdir = tempfile.mkdtemp( prefix="{}_{}".format( str(self)[:MAX_LEN_IDENTIFIER], date_str()), dir=self.local_dir) elif not os.path.exists(self.logdir): - os.makedirs(self.local_dir) + os.makedirs(self.logdir) self.result_logger = UnifiedLogger( self.config, @@ -424,7 +424,7 @@ def __getstate__(self): state["__logger_started__"] = True else: state["__logger_started__"] = False - return state + return copy.deepcopy(state) def __setstate__(self, state): logger_started = state.pop("__logger_started__") diff --git a/python/ray/tune/trial_executor.py b/python/ray/tune/trial_executor.py index 74f531985b2d1..921658d21918d 100644 --- a/python/ray/tune/trial_executor.py +++ b/python/ray/tune/trial_executor.py @@ -58,8 +58,7 @@ def try_checkpoint_metadata(self, trial): return try: logger.debug("Saving trial metadata.") - metadata = json.dumps(trial.__getstate__(), indent=2) - self._checkpoints[trial.trial_id] = metadata + self._checkpoints[trial.trial_id] = trial.__getstate__() except Exception: logger.exception("Error checkpointing trial metadata.") diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index 8b46bd6843541..b571d5dd91415 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -118,7 +118,7 @@ def checkpoint(self): } tmp_file_name = os.path.join(checkpoint_dir, ".tmp_checkpoint") with open(tmp_file_name, "w") as f: - json.dump(runner_state, f) + json.dump(runner_state, f, indent=2) os.rename(tmp_file_name, os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE)) @@ -152,10 +152,8 @@ def restore(cls, "r") as f: runner_state = json.load(f) - logger.warning( - "Tune recovery is still experimental. " - "There is limited search algorithm recovery support. " - "Restoring with a BasicVariantGenerator and FIFOScheduler.") + logger.warning("Tune recovery is still experimental. " + "There is limited search algorithm recovery support. ") from ray.tune.suggest import BasicVariantGenerator runner = TrialRunner( @@ -166,11 +164,8 @@ def restore(cls, runner.__setstate__(runner_state["runner_data"]) logger.info("Adding trials.") - trial_checkpoints = [ - json.loads(cp) for cp in runner_state["checkpoints"] - ] trials = [] - for trial_cp in trial_checkpoints: + for trial_cp in runner_state["checkpoints"]: new_trial = Trial(trial_cp["trainable_name"]) new_trial.__setstate__(trial_cp) trials += [new_trial] @@ -259,8 +254,7 @@ def add_trial(self, trial): """ trial.set_verbose(self._verbose) self._scheduler_alg.on_trial_add(self, trial) - if trial.status == Trial.PENDING: - self.trial_executor.try_checkpoint_metadata(trial) + self.trial_executor.try_checkpoint_metadata(trial) self._trials.append(trial) def debug_string(self, max_debug=MAX_DEBUG_TRIALS): diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index 7990df0ab05f9..997fa222dcdbf 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -118,7 +118,8 @@ def run_experiments(experiments=None, runner = TrialRunner.restore(checkpoint_dir, search_alg, scheduler, trial_executor) except Exception: - logger.info("Runner restore failed. Restarting experiment.") + logger.exception( + "Runner restore failed. Restarting experiment.") if not runner: if scheduler is None: From 677e09de614cb601d7500f6ceec307a2fb9cb738 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 26 Dec 2018 17:10:48 +0800 Subject: [PATCH 149/172] resume to prompt on None --- python/ray/tune/trial_executor.py | 1 - python/ray/tune/tune.py | 57 ++++++++++++++++++++++--------- 2 files changed, 40 insertions(+), 18 deletions(-) diff --git a/python/ray/tune/trial_executor.py b/python/ray/tune/trial_executor.py index 921658d21918d..f22f6f1c3ace4 100644 --- a/python/ray/tune/trial_executor.py +++ b/python/ray/tune/trial_executor.py @@ -3,7 +3,6 @@ from __future__ import division from __future__ import print_function -import json import logging from ray.tune.trial import Trial, Checkpoint diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index 997fa222dcdbf..1312002abe066 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -2,6 +2,7 @@ from __future__ import division from __future__ import print_function +import click import logging import os import time @@ -35,9 +36,24 @@ def _make_scheduler(args): def _find_checkpoint_dir(exp_list): - exp = exp_list[0] - # TODO(rliaw): Make sure this is resolved earlier. - return os.path.join(exp.spec["local_dir"], exp.name) + if exp_list: + exp = exp_list[0] + # TODO(rliaw): Make sure this is resolved earlier. + return os.path.join(exp.spec["local_dir"], exp.name) + else: + return None + + +def try_restore_runner(checkpoint_dir, search_alg, scheduler, trial_executor): + logger.warn("Restoring from previous experiment and " + "ignoring any new changes to specification.") + new_runner = None + try: + new_runner = TrialRunner.restore(checkpoint_dir, search_alg, scheduler, + trial_executor) + except Exception: + logger.exception("Runner restore failed. Restarting experiment.") + return new_runner def run_experiments(experiments=None, @@ -46,7 +62,7 @@ def run_experiments(experiments=None, with_server=False, server_port=TuneServer.DEFAULT_PORT, verbose=True, - resume=False, + resume=None, queue_trials=False, trial_executor=None, raise_on_failed_trial=True): @@ -66,9 +82,9 @@ def run_experiments(experiments=None, using the Client API. server_port (int): Port number for launching TuneServer. verbose (bool): How much output should be printed for each trial. - resume (bool): Turns on checkpointing. - If checkpoint exists, the experiment will resume from there. - Only the first checkpointable experiment local_dir is checked. + resume (bool|None): If checkpoint exists, the experiment will + resume from there. If resume is None, Tune will prompt if + checkpoint detected. queue_trials (bool): Whether to queue trials when the cluster does not currently have enough resources to launch one. This should be set to True when running on an autoscaling cluster to enable @@ -102,24 +118,31 @@ def run_experiments(experiments=None, # and it conducts the implicit registration. experiments = convert_to_experiment_list(experiments) checkpoint_dir = _find_checkpoint_dir(experiments) - + if checkpoint_dir: + logger.info("Using checkpoint dir: {}.".format(checkpoint_dir)) runner = None if resume: - logger.info("Using checkpoint dir: {}.".format(checkpoint_dir)) + if not checkpoint_dir: + raise ValueError( + "checkpoint_dir not detected. " + "Set resume=False or set a local_dir." + ) if not os.path.exists( os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE)): logger.warn( "Did not find checkpoint file in {}.".format(checkpoint_dir)) else: - logger.warn("Restoring from previous experiment and " - "ignoring any new changes to specification.") - try: - runner = TrialRunner.restore(checkpoint_dir, search_alg, - scheduler, trial_executor) - except Exception: - logger.exception( - "Runner restore failed. Restarting experiment.") + runner = try_restore_runner(checkpoint_dir, search_alg, scheduler, + trial_executor) + elif resume is None: + if os.path.exists(os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE)): + if click.confirm("Detected checkpoint dir: {}. Restore?".format( + checkpoint_dir)): + runner = try_restore_runner(checkpoint_dir, search_alg, + scheduler, trial_executor) + else: + logger.info("Overriding checkpoint and restarting experiment.") if not runner: if scheduler is None: From 83b152367e0cb9e0d0674bcd583837be2a8e3a25 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 26 Dec 2018 17:32:19 +0800 Subject: [PATCH 150/172] turn off prompt for tests --- python/ray/tune/test/cluster_tests.py | 2 ++ python/ray/tune/test/trial_runner_test.py | 6 ++++++ python/ray/tune/tune.py | 2 +- 3 files changed, 9 insertions(+), 1 deletion(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 6bed64940acf2..93fe61b7dfd51 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -62,6 +62,7 @@ def _start_new_cluster(): @pytest.fixture def start_connected_cluster(): # Start the Ray processes. + os.environ["TUNE_RESUME_PROMPT_OFF"] = "True" cluster = _start_new_cluster() yield cluster # The code after the yield will run as teardown code. @@ -73,6 +74,7 @@ def start_connected_cluster(): def start_connected_emptyhead_cluster(): """Starts head with no resources.""" + os.environ["TUNE_RESUME_PROMPT_OFF"] = "True" cluster = Cluster( initialize_head=True, connect=True, diff --git a/python/ray/tune/test/trial_runner_test.py b/python/ray/tune/test/trial_runner_test.py index cd2d3d45dcee6..45bfa11cce17d 100644 --- a/python/ray/tune/test/trial_runner_test.py +++ b/python/ray/tune/test/trial_runner_test.py @@ -38,6 +38,7 @@ class TrainableFunctionApiTest(unittest.TestCase): def setUp(self): + os.environ["TUNE_RESUME_PROMPT_OFF"] = "True" ray.init(num_cpus=4, num_gpus=0) def tearDown(self): @@ -543,6 +544,7 @@ def _restore(self, state): class RunExperimentTest(unittest.TestCase): def setUp(self): + os.environ["TUNE_RESUME_PROMPT_OFF"] = "True" ray.init() def tearDown(self): @@ -779,6 +781,7 @@ def sync_func(local, remote): class VariantGeneratorTest(unittest.TestCase): def setUp(self): + os.environ["TUNE_RESUME_PROMPT_OFF"] = "True" ray.init() def tearDown(self): @@ -968,6 +971,9 @@ def on_trial_complete(self, trial_id, error=False, **kwargs): class TrialRunnerTest(unittest.TestCase): + def setUp(self): + os.environ["TUNE_RESUME_PROMPT_OFF"] = "True" + def tearDown(self): ray.shutdown() _register_all() # re-register the evicted objects diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index 1312002abe066..660566d453f2f 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -135,7 +135,7 @@ def run_experiments(experiments=None, else: runner = try_restore_runner(checkpoint_dir, search_alg, scheduler, trial_executor) - elif resume is None: + elif resume is None and not os.environ.get("TUNE_RESUME_PROMPT_OFF"): if os.path.exists(os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE)): if click.confirm("Detected checkpoint dir: {}. Restore?".format( checkpoint_dir)): From d1180c3ad41c48f2eaea7524c8a8c2a32b157a9d Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 26 Dec 2018 01:36:21 -0800 Subject: [PATCH 151/172] Merge --- python/ray/tune/tune.py | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index 660566d453f2f..a8215c1af857f 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -124,10 +124,8 @@ def run_experiments(experiments=None, if resume: if not checkpoint_dir: - raise ValueError( - "checkpoint_dir not detected. " - "Set resume=False or set a local_dir." - ) + raise ValueError("checkpoint_dir not detected. " + "Set resume=False or set a local_dir.") if not os.path.exists( os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE)): logger.warn( @@ -136,7 +134,8 @@ def run_experiments(experiments=None, runner = try_restore_runner(checkpoint_dir, search_alg, scheduler, trial_executor) elif resume is None and not os.environ.get("TUNE_RESUME_PROMPT_OFF"): - if os.path.exists(os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE)): + if checkpoint_dir and os.path.exists( + os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE)): if click.confirm("Detected checkpoint dir: {}. Restore?".format( checkpoint_dir)): runner = try_restore_runner(checkpoint_dir, search_alg, From 45d5d903c082b02d587ecf577c008d2a89bc0939 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 26 Dec 2018 17:46:33 +0800 Subject: [PATCH 152/172] fix for tests --- python/ray/tune/test/trial_scheduler_test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/ray/tune/test/trial_scheduler_test.py b/python/ray/tune/test/trial_scheduler_test.py index e8e2f20544b49..9d463850f1438 100644 --- a/python/ray/tune/test/trial_scheduler_test.py +++ b/python/ray/tune/test/trial_scheduler_test.py @@ -578,6 +578,7 @@ def __init__(self, i, config): self.logger_running = False self.restored_checkpoint = None self.resources = Resources(1, 0) + self.trial_name = None class PopulationBasedTestingSuite(unittest.TestCase): From b89b910d09383c2310023023924a78a2c05ec10f Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 26 Dec 2018 01:52:30 -0800 Subject: [PATCH 153/172] Accidentally removed a file --- python/ray/tune/examples/hyperband_example.py | 74 +++++++++++++++++++ 1 file changed, 74 insertions(+) create mode 100755 python/ray/tune/examples/hyperband_example.py diff --git a/python/ray/tune/examples/hyperband_example.py b/python/ray/tune/examples/hyperband_example.py new file mode 100755 index 0000000000000..d403a0e0f8af1 --- /dev/null +++ b/python/ray/tune/examples/hyperband_example.py @@ -0,0 +1,74 @@ +#!/usr/bin/env python + +from __future__ import absolute_import +from __future__ import division +from __future__ import print_function + +import argparse +import json +import os +import random + +import numpy as np + +import ray +from ray.tune import Trainable, run_experiments, Experiment, sample_from +from ray.tune.schedulers import HyperBandScheduler + + +class MyTrainableClass(Trainable): + """Example agent whose learning curve is a random sigmoid. + + The dummy hyperparameters "width" and "height" determine the slope and + maximum reward value reached. + """ + + def _setup(self, config): + self.timestep = 0 + + def _train(self): + self.timestep += 1 + v = np.tanh(float(self.timestep) / self.config["width"]) + v *= self.config["height"] + + # Here we use `episode_reward_mean`, but you can also report other + # objectives such as loss or accuracy. + return {"episode_reward_mean": v} + + def _save(self, checkpoint_dir): + path = os.path.join(checkpoint_dir, "checkpoint") + with open(path, "w") as f: + f.write(json.dumps({"timestep": self.timestep})) + return path + + def _restore(self, checkpoint_path): + with open(checkpoint_path) as f: + self.timestep = json.loads(f.read())["timestep"] + + +if __name__ == "__main__": + parser = argparse.ArgumentParser() + parser.add_argument( + "--smoke-test", action="store_true", help="Finish quickly for testing") + args, _ = parser.parse_known_args() + ray.init() + + # Hyperband early stopping, configured with `episode_reward_mean` as the + # objective and `training_iteration` as the time unit, + # which is automatically filled by Tune. + hyperband = HyperBandScheduler( + time_attr="training_iteration", + reward_attr="episode_reward_mean", + max_t=100) + + exp = Experiment( + name="hyperband_test", + run=MyTrainableClass, + num_samples=20, + stop={"training_iteration": 1 if args.smoke_test else 99999}, + config={ + "width": sample_from(lambda spec: 10 + int(90 * random.random())), + "height": sample_from(lambda spec: int(100 * random.random())) + }) + + run_experiments(exp, scheduler=hyperband) From 6995a596973b92c254efeac4b8be223e68d60dee Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 26 Dec 2018 01:54:12 -0800 Subject: [PATCH 154/172] example not needed --- .../ray/tune/examples/checkpoint_example.py | 68 ------------------- 1 file changed, 68 deletions(-) delete mode 100755 python/ray/tune/examples/checkpoint_example.py diff --git a/python/ray/tune/examples/checkpoint_example.py b/python/ray/tune/examples/checkpoint_example.py deleted file mode 100755 index 3ed6c2a5e380f..0000000000000 --- a/python/ray/tune/examples/checkpoint_example.py +++ /dev/null @@ -1,68 +0,0 @@ -#!/usr/bin/env python - -from __future__ import absolute_import -from __future__ import division -from __future__ import print_function - -import argparse -import json -import os -import random -import time - -import numpy as np - -import ray -from ray.tune import Trainable, run_experiments, Experiment, sample_from - - -class MyTrainableClass(Trainable): - """Example agent whose learning curve is a random sigmoid. - - The dummy hyperparameters "width" and "height" determine the slope and - maximum reward value reached. - """ - - def _setup(self, config): - self.timestep = 0 - - def _train(self): - self.timestep += 1 - v = np.tanh(float(self.timestep) / self.config["width"]) - v *= self.config["height"] - time.sleep(1) - # Here we use `episode_reward_mean`, but you can also report other - # objectives such as loss or accuracy. - return {"episode_reward_mean": v} - - def _save(self, checkpoint_dir): - path = os.path.join(checkpoint_dir, "checkpoint") - with open(path, "w") as f: - f.write(json.dumps({"timestep": self.timestep})) - return path - - def _restore(self, checkpoint_path): - with open(checkpoint_path) as f: - self.timestep = json.loads(f.read())["timestep"] - - -if __name__ == "__main__": - parser = argparse.ArgumentParser() - parser.add_argument( - "--smoke-test", action="store_true", help="Finish quickly for testing") - args, _ = parser.parse_known_args() - ray.init(num_cpus=4) - - exp = Experiment( - name="hyperband_test", - run=MyTrainableClass, - num_samples=20, - stop={"training_iteration": 50}, - checkpoint_freq=4, - local_dir="~/ray_results/checkpoint_test2/", - config={ - "width": sample_from(lambda spec: 10 + int(90 * random.random())), - "height": sample_from(lambda spec: int(100 * random.random())) - }) - - run_experiments(exp, resume=True) From 86b37f66d43e50e9aea8e59f1bd23f91ab496c52 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 26 Dec 2018 01:55:41 -0800 Subject: [PATCH 155/172] no need to change this --- python/ray/tune/log_sync.py | 28 ++++++++++++++-------------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/python/ray/tune/log_sync.py b/python/ray/tune/log_sync.py index 70bb80df4df40..2046165c0129d 100644 --- a/python/ray/tune/log_sync.py +++ b/python/ray/tune/log_sync.py @@ -30,20 +30,6 @@ ALLOWED_REMOTE_PREFIXES = (S3_PREFIX, GCS_PREFIX) -def validate_sync_function(sync_function): - if sync_function is None: - return - elif isinstance(sync_function, str): - assert "{remote_dir}" in sync_function, ( - "Sync template missing '{remote_dir}'.") - assert "{local_dir}" in sync_function, ( - "Sync template missing '{local_dir}'.") - elif not (isinstance(sync_function, types.FunctionType) - or isinstance(sync_function, tune_function)): - raise ValueError("Sync function {} must be string or function".format( - sync_function)) - - def get_syncer(local_dir, remote_dir=None, sync_function=None): if remote_dir: if not sync_function and not any( @@ -79,6 +65,20 @@ def wait_for_log_sync(): syncer.wait() +def validate_sync_function(sync_function): + if sync_function is None: + return + elif isinstance(sync_function, str): + assert "{remote_dir}" in sync_function, ( + "Sync template missing '{remote_dir}'.") + assert "{local_dir}" in sync_function, ( + "Sync template missing '{local_dir}'.") + elif not (isinstance(sync_function, types.FunctionType) + or isinstance(sync_function, tune_function)): + raise ValueError("Sync function {} must be string or function".format( + sync_function)) + + class _LogSyncer(object): """Log syncer for tune. From f92198e07a2987f2c1dcb136386f360e5a625aff Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 26 Dec 2018 20:16:49 +0800 Subject: [PATCH 156/172] doc changes and small guard --- doc/source/tune-usage.rst | 18 +++--------------- python/ray/tune/trainable.py | 3 ++- 2 files changed, 5 insertions(+), 16 deletions(-) diff --git a/doc/source/tune-usage.rst b/doc/source/tune-usage.rst index 0291ada805cd8..f86ecc3b6712e 100644 --- a/doc/source/tune-usage.rst +++ b/doc/source/tune-usage.rst @@ -299,9 +299,9 @@ of a trial, you can additionally set the checkpoint_at_end to True. An example i Recovering From Failures (Experimental) ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ -Tune automatically persists the progress of your experiments, so if an experiment crashes or is otherwise cancelled, it can be resumed by passing `resume=True` in `run_experiments()`. If resume is not enabled, a new experiment will be created instead. You can always force a new experiment to be created by changing the experiment name. +Tune automatically persists the progress of your experiments, so if an experiment crashes or is otherwise cancelled, it can be resumed after prompting. Prompting can be turned off with ``resume=True``. If ``resume=False``, a new experiment will be created instead. You can always force a new experiment to be created by changing the experiment name. -Note that trials will be restored to their last checkpoint. If checkpointing is not enabled, unfinished trials will be restarted from scratch. +Note that trials will be restored to their last checkpoint. If trial checkpointing is not enabled, unfinished trials will be restarted from scratch. E.g.: @@ -316,19 +316,7 @@ E.g.: }, resume=True) -To restore from an experiment, e.g.: - -.. code-block:: python - - run_experiments({ - "my_experiment_name": { - "run": my_trainable - "checkpoint_freq": 10, - "local_dir": "~/path/to/results" - }, - }, resume=True) - -This will restore the entire experiment state from the ``local_dir/my_experiment_name`` of the first checkpointable experiment provided. Importantly, note that any changes to the experiment specification upon resume will be ignored. +Upon a second run, this will restore the entire experiment state from ``~/path/to/results/my_experiment_name``. Importantly, any changes to the experiment specification upon resume will be ignored. This feature is still experimental, so any provided Trial Scheduler or Search Algorithm will not be preserved. Only ``FIFOScheduler`` and ``BasicVariantGenerator`` will be supported. diff --git a/python/ray/tune/trainable.py b/python/ray/tune/trainable.py index 5824c5221ff5b..dfcdf530f70c9 100644 --- a/python/ray/tune/trainable.py +++ b/python/ray/tune/trainable.py @@ -216,7 +216,8 @@ def save(self, checkpoint_dir=None): checkpoint_dir = os.path.join(checkpoint_dir or self.logdir, "checkpoint_{}".format(self._iteration)) - os.makedirs(checkpoint_dir) + if not os.path.exists(checkpoint_dir): + os.makedirs(checkpoint_dir) checkpoint = self._save(checkpoint_dir) saved_as_dict = False if isinstance(checkpoint, str): From a09677217930b5b801c3799cc0b124c8969aa355 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 26 Dec 2018 20:36:12 +0800 Subject: [PATCH 157/172] renames, remove checkpoint_mode --- python/ray/tune/ray_trial_executor.py | 3 +-- python/ray/tune/test/cluster_tests.py | 8 +++---- python/ray/tune/trial.py | 18 +++++++------- python/ray/tune/trial_executor.py | 24 ++++++++----------- python/ray/tune/trial_runner.py | 34 ++++++++++++--------------- python/ray/tune/tune.py | 11 +++++---- 6 files changed, 44 insertions(+), 54 deletions(-) diff --git a/python/ray/tune/ray_trial_executor.py b/python/ray/tune/ray_trial_executor.py index 5e0a7f5660c2a..3706d00f0a520 100644 --- a/python/ray/tune/ray_trial_executor.py +++ b/python/ray/tune/ray_trial_executor.py @@ -20,8 +20,7 @@ class RayTrialExecutor(TrialExecutor): """An implemention of TrialExecutor based on Ray.""" def __init__(self, queue_trials=False): - super(RayTrialExecutor, self).__init__( - queue_trials, checkpoint_mode=True) + super(RayTrialExecutor, self).__init__(queue_trials) self._running = {} # Since trial resume after paused should not run # trial.train.remote(), thus no more new remote object id generated. diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 93fe61b7dfd51..fcb0d1b829705 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -379,7 +379,7 @@ def test_cluster_rllib_restore(start_connected_cluster, tmpdir): # The trainable returns every 0.5 seconds, so this should not miss # the checkpoint. for i in range(50): - if os.path.exists(os.path.join(dirpath, TrialRunner.CKPT_FILE)): + if os.path.exists(os.path.join(dirpath, TrialRunner.CKPT_FILE_NAME)): # Inspect the internal trialrunner runner = TrialRunner.restore(dirpath) trials = runner.get_trials() @@ -388,7 +388,7 @@ def test_cluster_rllib_restore(start_connected_cluster, tmpdir): break time.sleep(0.2) - if not os.path.exists(os.path.join(dirpath, TrialRunner.CKPT_FILE)): + if not os.path.exists(os.path.join(dirpath, TrialRunner.CKPT_FILE_NAME)): raise RuntimeError("Checkpoint file didn't appear.") ray.shutdown() @@ -448,7 +448,7 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): # The trainable returns every 0.5 seconds, so this should not miss # the checkpoint. for i in range(50): - if os.path.exists(os.path.join(dirpath, TrialRunner.CKPT_FILE)): + if os.path.exists(os.path.join(dirpath, TrialRunner.CKPT_FILE_NAME)): # Inspect the internal trialrunner runner = TrialRunner.restore(dirpath) trials = runner.get_trials() @@ -457,7 +457,7 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): break time.sleep(0.2) - if not os.path.exists(os.path.join(dirpath, TrialRunner.CKPT_FILE)): + if not os.path.exists(os.path.join(dirpath, TrialRunner.CKPT_FILE_NAME)): raise RuntimeError("Checkpoint file didn't appear.") ray.shutdown() diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index c2f5e72e4c3bd..6991bbd781de1 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -398,22 +398,20 @@ def __getstate__(self): Sets RUNNING trials to PENDING, and flushes the result logger. Note this can only occur if the trial holds a DISK checkpoint. """ - if not self._checkpoint.storage == Checkpoint.DISK: - raise ValueError("Checkpoint cannot be in-memory.") + assert self._checkpoint.storage == Checkpoint.DISK, ( + "Checkpoint must not be in-memory.") state = self.__dict__.copy() + state["resources"] = resources_to_json(self.resources), pickle_data = { "_checkpoint": self._checkpoint, "config": self.config, "custom_loggers": self.custom_loggers, - "resources": resources_to_json(self.resources), "sync_function": self.sync_function } - state["__data__"] = binary_to_hex(cloudpickle.dumps(pickle_data)) - # Remove the unpicklable entries. - for key in pickle_data: - state[key] = None + for key, value in pickle_data.items(): + state[key] = binary_to_hex(cloudpickle.dumps(value)) state["runner"] = None state["result_logger"] = None @@ -428,9 +426,9 @@ def __getstate__(self): def __setstate__(self, state): logger_started = state.pop("__logger_started__") - other_data = cloudpickle.loads(hex_to_binary(state.pop("__data__"))) - other_data["resources"] = json_to_resources(other_data["resources"]) - state.update(other_data) + state["resources"] = json_to_resources(state["resources"]) + for key in ["_checkpoint", "config", "custom_loggers", "sync_function"]: + state[key] = cloudpickle.loads(hex_to_binary(state[key])) self.__dict__.update(state) Trial._registration_check(self.trainable_name) diff --git a/python/ray/tune/trial_executor.py b/python/ray/tune/trial_executor.py index f22f6f1c3ace4..d87ca3228b8d3 100644 --- a/python/ray/tune/trial_executor.py +++ b/python/ray/tune/trial_executor.py @@ -15,7 +15,7 @@ class TrialExecutor(object): and starting/stopping trials. """ - def __init__(self, queue_trials=False, checkpoint_mode=False): + def __init__(self, queue_trials=False): """Initializes a new TrialExecutor. Args: @@ -23,11 +23,8 @@ def __init__(self, queue_trials=False, checkpoint_mode=False): not currently have enough resources to launch one. This should be set to True when running on an autoscaling cluster to enable automatic scale-up. - checkpoint_mode (bool): Whether to track metadata on status - change. """ self._queue_trials = queue_trials - self._checkpoint_mode = checkpoint_mode self._checkpoints = {} def set_status(self, trial, status): @@ -46,20 +43,19 @@ def set_status(self, trial, status): self.try_checkpoint_metadata(trial) def try_checkpoint_metadata(self, trial): - """Checkpoints metadata if checkpoint_mode is True. + """Checkpoints metadata. Args: trial (Trial): Trial to checkpoint. """ - if self._checkpoint_mode: - if trial._checkpoint.storage == Checkpoint.MEMORY: - logger.debug("Not saving data for trial w/ memory checkpoint.") - return - try: - logger.debug("Saving trial metadata.") - self._checkpoints[trial.trial_id] = trial.__getstate__() - except Exception: - logger.exception("Error checkpointing trial metadata.") + if trial._checkpoint.storage == Checkpoint.MEMORY: + logger.debug("Not saving data for trial w/ memory checkpoint.") + return + try: + logger.debug("Saving trial metadata.") + self._checkpoints[trial.trial_id] = trial.__getstate__() + except Exception: + logger.exception("Error checkpointing trial metadata.") def get_checkpoints(self): """Returns a copy of mapping of the trial ID to pickled metadata.""" diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index b571d5dd91415..c53a1eb52948b 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -50,13 +50,13 @@ class TrialRunner(object): misleading benchmark results. """ - CKPT_FILE = "experiment.state" + CKPT_FILE_NAME = "experiment_state.json" def __init__(self, search_alg, scheduler=None, launch_web_server=False, - checkpoint_dir=None, + metadata_checkpoint_dir=None, server_port=TuneServer.DEFAULT_PORT, verbose=True, queue_trials=False, @@ -68,7 +68,7 @@ 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 + metadata_checkpoint_dir (str): Path where global checkpoints are stored and restored from. server_port (int): Port number for launching TuneServer verbose (bool): Flag for verbosity. If False, trial results @@ -100,33 +100,31 @@ def __init__(self, self._trials = [] self._stop_queue = [] - self._checkpoint_dir = checkpoint_dir + self._metadata_checkpoint_dir = metadata_checkpoint_dir def checkpoint(self): - """Saves execution state to `self._checkpoint_dir` if provided.""" - if not self._checkpoint_dir: + """Saves execution state to `self._metadata_checkpoint_dir` if provided.""" + if not self._metadata_checkpoint_dir: return - checkpoint_dir = self._checkpoint_dir - if not os.path.exists(checkpoint_dir): - logger.debug("Checkpoint directory newly created.") - logger.warning("Search Algorithm and Scheduler not checkpointed.") - os.makedirs(checkpoint_dir) + metadata_checkpoint_dir = self._metadata_checkpoint_dir + if not os.path.exists(metadata_checkpoint_dir): + os.makedirs(metadata_checkpoint_dir) runner_state = { "checkpoints": list( self.trial_executor.get_checkpoints().values()), "runner_data": self.__getstate__() } - tmp_file_name = os.path.join(checkpoint_dir, ".tmp_checkpoint") + tmp_file_name = os.path.join(metadata_checkpoint_dir, ".tmp_checkpoint") with open(tmp_file_name, "w") as f: json.dump(runner_state, f, indent=2) os.rename(tmp_file_name, - os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE)) - return checkpoint_dir + os.path.join(metadata_checkpoint_dir, TrialRunner.CKPT_FILE_NAME)) + return metadata_checkpoint_dir @classmethod def restore(cls, - checkpoint_dir, + metadata_checkpoint_dir, search_alg=None, scheduler=None, trial_executor=None): @@ -135,10 +133,8 @@ def restore(cls, Requires user to manually re-register their objects. Also stops all ongoing trials. - TODO: Consider checkpointing registry too. - Args: - checkpoint_dir (str): Path to checkpoint (previously specified). + metadata_checkpoint_dir (str): Path to checkpoint (previously specified). search_alg (SearchAlgorithm): Search Algorithm. Defaults to BasicVariantGenerator. scheduler (TrialScheduler): Scheduler for executing @@ -148,7 +144,7 @@ def restore(cls, Returns: runner (TrialRunner): A TrialRunner to resume experiments from. """ - with open(os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE), + with open(os.path.join(metadata_checkpoint_dir, TrialRunner.CKPT_FILE_NAME), "r") as f: runner_state = json.load(f) diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index a8215c1af857f..c5bcdb79a9a88 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -124,18 +124,19 @@ def run_experiments(experiments=None, if resume: if not checkpoint_dir: - raise ValueError("checkpoint_dir not detected. " - "Set resume=False or set a local_dir.") + raise ValueError( + "checkpoint_dir not detected. " + "Set resume=False or set a local_dir." + ) if not os.path.exists( - os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE)): + os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE_NAME)): logger.warn( "Did not find checkpoint file in {}.".format(checkpoint_dir)) else: runner = try_restore_runner(checkpoint_dir, search_alg, scheduler, trial_executor) elif resume is None and not os.environ.get("TUNE_RESUME_PROMPT_OFF"): - if checkpoint_dir and os.path.exists( - os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE)): + if os.path.exists(os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE_NAME)): if click.confirm("Detected checkpoint dir: {}. Restore?".format( checkpoint_dir)): runner = try_restore_runner(checkpoint_dir, search_alg, From ec726d2cc5b79bdf6fbb9d481158a69cb05a76f0 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Wed, 26 Dec 2018 04:37:05 -0800 Subject: [PATCH 158/172] Update python/ray/tune/trial_runner.py Co-Authored-By: richardliaw --- python/ray/tune/trial_runner.py | 1 - 1 file changed, 1 deletion(-) diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index c53a1eb52948b..8430c868df9fe 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -159,7 +159,6 @@ def restore(cls, runner.__setstate__(runner_state["runner_data"]) - logger.info("Adding trials.") trials = [] for trial_cp in runner_state["checkpoints"]: new_trial = Trial(trial_cp["trainable_name"]) From bf45f4268ae706f9e8e10b5bcd4bbdfae58bbcc9 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 26 Dec 2018 22:23:09 +0800 Subject: [PATCH 159/172] Simpify, fix tests, address comments, make expmt mandatory --- python/ray/tune/test/cluster_tests.py | 3 +- python/ray/tune/test/trial_runner_test.py | 29 ++--------- python/ray/tune/trial.py | 4 +- python/ray/tune/trial_executor.py | 6 +-- python/ray/tune/trial_runner.py | 30 +++++++----- python/ray/tune/tune.py | 60 ++++++++++------------- 6 files changed, 56 insertions(+), 76 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index fcb0d1b829705..144697cdce653 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -276,7 +276,8 @@ def test_cluster_down_simple(start_connected_cluster, tmpdir): assert cluster.wait_for_nodes() dirpath = str(tmpdir) - runner = TrialRunner(BasicVariantGenerator(), checkpoint_dir=dirpath) + runner = TrialRunner( + BasicVariantGenerator(), metadata_checkpoint_dir=dirpath) kwargs = { "stopping_criterion": { "training_iteration": 2 diff --git a/python/ray/tune/test/trial_runner_test.py b/python/ray/tune/test/trial_runner_test.py index 45bfa11cce17d..8c47fc48d3449 100644 --- a/python/ray/tune/test/trial_runner_test.py +++ b/python/ray/tune/test/trial_runner_test.py @@ -617,29 +617,6 @@ def train(config, reporter): self.assertEqual(trial.status, Trial.TERMINATED) self.assertEqual(trial.last_result[TIMESTEPS_TOTAL], 99) - def testSpecifyAlgorithm(self): - """Tests run_experiments works without specifying experiment.""" - - def train(config, reporter): - for i in range(100): - reporter(timesteps_total=i) - - register_trainable("f1", train) - - alg = BasicVariantGenerator() - alg.add_configurations({ - "foo": { - "run": "f1", - "config": { - "script_min_iter_time_s": 0 - } - } - }) - trials = run_experiments(search_alg=alg) - for trial in trials: - self.assertEqual(trial.status, Trial.TERMINATED) - self.assertEqual(trial.last_result[TIMESTEPS_TOTAL], 99) - def testAutoregisterTrainable(self): def train(config, reporter): for i in range(100): @@ -1663,7 +1640,8 @@ def testTrialSaveRestore(self): ray.init(num_cpus=3) tmpdir = tempfile.mkdtemp() - runner = TrialRunner(BasicVariantGenerator(), checkpoint_dir=tmpdir) + runner = TrialRunner( + BasicVariantGenerator(), metadata_checkpoint_dir=tmpdir) trials = [ Trial( "__fake", @@ -1722,7 +1700,8 @@ def testTrialNoSave(self): ray.init(num_cpus=3) tmpdir = tempfile.mkdtemp() - runner = TrialRunner(BasicVariantGenerator(), checkpoint_dir=tmpdir) + runner = TrialRunner( + BasicVariantGenerator(), metadata_checkpoint_dir=tmpdir) runner.add_trial( Trial( diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index 6991bbd781de1..24ae67baf5876 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -427,7 +427,9 @@ def __getstate__(self): def __setstate__(self, state): logger_started = state.pop("__logger_started__") state["resources"] = json_to_resources(state["resources"]) - for key in ["_checkpoint", "config", "custom_loggers", "sync_function"]: + for key in [ + "_checkpoint", "config", "custom_loggers", "sync_function" + ]: state[key] = cloudpickle.loads(hex_to_binary(state[key])) self.__dict__.update(state) diff --git a/python/ray/tune/trial_executor.py b/python/ray/tune/trial_executor.py index d87ca3228b8d3..22d6d85eb78d0 100644 --- a/python/ray/tune/trial_executor.py +++ b/python/ray/tune/trial_executor.py @@ -25,7 +25,7 @@ def __init__(self, queue_trials=False): automatic scale-up. """ self._queue_trials = queue_trials - self._checkpoints = {} + self._cached_trial_state = {} def set_status(self, trial, status): """Sets status and checkpoints metadata if needed. @@ -53,13 +53,13 @@ def try_checkpoint_metadata(self, trial): return try: logger.debug("Saving trial metadata.") - self._checkpoints[trial.trial_id] = trial.__getstate__() + self._cached_trial_state[trial.trial_id] = trial.__getstate__() except Exception: 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() + return self._cached_trial_state.copy() def has_resources(self, resources): """Returns whether this runner has at least the specified resources.""" diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index c53a1eb52948b..3af839b2ab74b 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -68,8 +68,8 @@ def __init__(self, Trial objects. scheduler (TrialScheduler): Defaults to FIFOScheduler. launch_web_server (bool): Flag for starting TuneServer - metadata_checkpoint_dir (str): Path where global checkpoints are stored - and restored from. + metadata_checkpoint_dir (str): Path where + global checkpoints are stored and restored from. server_port (int): Port number for launching TuneServer verbose (bool): Flag for verbosity. If False, trial results will not be output. @@ -103,7 +103,7 @@ def __init__(self, self._metadata_checkpoint_dir = metadata_checkpoint_dir def checkpoint(self): - """Saves execution state to `self._metadata_checkpoint_dir` if provided.""" + """Saves execution state to `self._metadata_checkpoint_dir`.""" if not self._metadata_checkpoint_dir: return metadata_checkpoint_dir = self._metadata_checkpoint_dir @@ -114,12 +114,14 @@ def checkpoint(self): self.trial_executor.get_checkpoints().values()), "runner_data": self.__getstate__() } - tmp_file_name = os.path.join(metadata_checkpoint_dir, ".tmp_checkpoint") + tmp_file_name = os.path.join(metadata_checkpoint_dir, + ".tmp_checkpoint") with open(tmp_file_name, "w") as f: json.dump(runner_state, f, indent=2) - os.rename(tmp_file_name, - os.path.join(metadata_checkpoint_dir, TrialRunner.CKPT_FILE_NAME)) + os.rename( + tmp_file_name, + os.path.join(metadata_checkpoint_dir, TrialRunner.CKPT_FILE_NAME)) return metadata_checkpoint_dir @classmethod @@ -134,7 +136,7 @@ def restore(cls, all ongoing trials. Args: - metadata_checkpoint_dir (str): Path to checkpoint (previously specified). + metadata_checkpoint_dir (str): Path to metadata checkpoints. search_alg (SearchAlgorithm): Search Algorithm. Defaults to BasicVariantGenerator. scheduler (TrialScheduler): Scheduler for executing @@ -144,12 +146,18 @@ def restore(cls, Returns: runner (TrialRunner): A TrialRunner to resume experiments from. """ - with open(os.path.join(metadata_checkpoint_dir, TrialRunner.CKPT_FILE_NAME), - "r") as f: + with open( + os.path.join(metadata_checkpoint_dir, + TrialRunner.CKPT_FILE_NAME), "r") as f: runner_state = json.load(f) - logger.warning("Tune recovery is still experimental. " - "There is limited search algorithm recovery support. ") + logger.warning("".join([ + "Attempting to resume experiment from {}. ".format( + metadata_checkpoint_dir), + "This feature is experimental, " + "and may not work with all search algorithms. ", + "This will ignore any new changes to specification." + ])) from ray.tune.suggest import BasicVariantGenerator runner = TrialRunner( diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index c5bcdb79a9a88..3af36cb62145f 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -36,17 +36,13 @@ def _make_scheduler(args): def _find_checkpoint_dir(exp_list): - if exp_list: - exp = exp_list[0] - # TODO(rliaw): Make sure this is resolved earlier. - return os.path.join(exp.spec["local_dir"], exp.name) - else: - return None + assert exp_list, "Experiments must be specified via `run_experiments`" + exp = exp_list[0] + # TODO(rliaw): Make sure this is resolved earlier. + return os.path.join(exp.spec["local_dir"], exp.name) def try_restore_runner(checkpoint_dir, search_alg, scheduler, trial_executor): - logger.warn("Restoring from previous experiment and " - "ignoring any new changes to specification.") new_runner = None try: new_runner = TrialRunner.restore(checkpoint_dir, search_alg, scheduler, @@ -56,7 +52,7 @@ def try_restore_runner(checkpoint_dir, search_alg, scheduler, trial_executor): return new_runner -def run_experiments(experiments=None, +def run_experiments(experiments, search_alg=None, scheduler=None, with_server=False, @@ -76,8 +72,6 @@ def run_experiments(experiments=None, scheduler (TrialScheduler): Scheduler for executing the experiment. Choose among FIFO (default), MedianStopping, AsyncHyperBand, and HyperBand. - checkpoint_dir (str): Path at which experiment checkpoints are stored - and restored from. with_server (bool): Starts a background Tune server. Needed for using the Client API. server_port (int): Port number for launching TuneServer. @@ -118,31 +112,27 @@ def run_experiments(experiments=None, # and it conducts the implicit registration. experiments = convert_to_experiment_list(experiments) checkpoint_dir = _find_checkpoint_dir(experiments) - if checkpoint_dir: - logger.info("Using checkpoint dir: {}.".format(checkpoint_dir)) + runner = None + restore = False + + if os.path.exists( + os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE_NAME)): + if resume: + restore = True + elif resume is None and not os.environ.get("TUNE_RESUME_PROMPT_OFF"): + msg = "Would you like to resume your experiment from '{}'?".format( + checkpoint_dir) + restore = click.confirm(msg, default=True) + else: + logger.info( + "Did not find checkpoint file in {}.".format(checkpoint_dir)) - if resume: - if not checkpoint_dir: - raise ValueError( - "checkpoint_dir not detected. " - "Set resume=False or set a local_dir." - ) - if not os.path.exists( - os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE_NAME)): - logger.warn( - "Did not find checkpoint file in {}.".format(checkpoint_dir)) - else: - runner = try_restore_runner(checkpoint_dir, search_alg, scheduler, - trial_executor) - elif resume is None and not os.environ.get("TUNE_RESUME_PROMPT_OFF"): - if os.path.exists(os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE_NAME)): - if click.confirm("Detected checkpoint dir: {}. Restore?".format( - checkpoint_dir)): - runner = try_restore_runner(checkpoint_dir, search_alg, - scheduler, trial_executor) - else: - logger.info("Overriding checkpoint and restarting experiment.") + if restore: + runner = try_restore_runner(checkpoint_dir, search_alg, scheduler, + trial_executor) + else: + logger.info("Starting a new experiment.") if not runner: if scheduler is None: @@ -156,7 +146,7 @@ def run_experiments(experiments=None, runner = TrialRunner( search_alg, scheduler=scheduler, - checkpoint_dir=checkpoint_dir, + metadata_checkpoint_dir=checkpoint_dir, launch_web_server=with_server, server_port=server_port, verbose=verbose, From 3f2e1ae6f320cea53ec1b0f7c4b8b7417f9e1eed Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 26 Dec 2018 22:43:13 +0800 Subject: [PATCH 160/172] typo --- python/ray/tune/logger.py | 1 - python/ray/tune/trial.py | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/python/ray/tune/logger.py b/python/ray/tune/logger.py index f689218fca785..3341c3c7601a0 100644 --- a/python/ray/tune/logger.py +++ b/python/ray/tune/logger.py @@ -178,7 +178,6 @@ 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): diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index 24ae67baf5876..747ff426f0e84 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -401,7 +401,7 @@ def __getstate__(self): assert self._checkpoint.storage == Checkpoint.DISK, ( "Checkpoint must not be in-memory.") state = self.__dict__.copy() - state["resources"] = resources_to_json(self.resources), + state["resources"] = resources_to_json(self.resources) pickle_data = { "_checkpoint": self._checkpoint, From e7b4f20ef9b1cc5d91ffc1b23c1d18e22ea96dfa Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 26 Dec 2018 23:08:40 +0800 Subject: [PATCH 161/172] some more tests --- doc/source/tune-usage.rst | 2 +- python/ray/tune/experiment.py | 1 + python/ray/tune/test/trial_runner_test.py | 1 + 3 files changed, 3 insertions(+), 1 deletion(-) diff --git a/doc/source/tune-usage.rst b/doc/source/tune-usage.rst index f86ecc3b6712e..0cd6572d44fc5 100644 --- a/doc/source/tune-usage.rst +++ b/doc/source/tune-usage.rst @@ -299,7 +299,7 @@ of a trial, you can additionally set the checkpoint_at_end to True. An example i Recovering From Failures (Experimental) ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ -Tune automatically persists the progress of your experiments, so if an experiment crashes or is otherwise cancelled, it can be resumed after prompting. Prompting can be turned off with ``resume=True``. If ``resume=False``, a new experiment will be created instead. You can always force a new experiment to be created by changing the experiment name. +Tune automatically persists the progress of your experiments, so if an experiment crashes or is otherwise cancelled, it can be resumed after prompting. The default setting of `resume=None` will cause Tune to prompt you for whether you want to resume. Prompting can be turned off with ``resume=True``. If ``resume=False``, a new experiment will be created instead. You can always force a new experiment to be created by changing the experiment name. Note that trials will be restored to their last checkpoint. If trial checkpointing is not enabled, unfinished trials will be restarted from scratch. diff --git a/python/ray/tune/experiment.py b/python/ray/tune/experiment.py index 7e08899b91440..4471edd2b3115 100644 --- a/python/ray/tune/experiment.py +++ b/python/ray/tune/experiment.py @@ -99,6 +99,7 @@ class Experiment(object): >>> num_samples=10, >>> local_dir="~/ray_results", >>> upload_dir="s3://your_bucket/path", + >>> checkpoint_freq=10, >>> max_failures=2) """ diff --git a/python/ray/tune/test/trial_runner_test.py b/python/ray/tune/test/trial_runner_test.py index 8c47fc48d3449..2faf30c1e2782 100644 --- a/python/ray/tune/test/trial_runner_test.py +++ b/python/ray/tune/test/trial_runner_test.py @@ -1741,6 +1741,7 @@ def testTrialNoSave(self): self.assertTrue( runner2.get_trial("checkpoint").status == Trial.TERMINATED) self.assertTrue(runner2.get_trial("pending").status == Trial.PENDING) + self.assertTrue(runner2.get_trial("pending").last_result is None) runner2.step() shutil.rmtree(tmpdir) From 80ace1d190007a0d33ca7cea67f03ffbc58a0b08 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Thu, 27 Dec 2018 10:16:10 +0800 Subject: [PATCH 162/172] lint --- python/ray/tune/trial.py | 2 +- python/ray/tune/trial_runner.py | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/python/ray/tune/trial.py b/python/ray/tune/trial.py index 747ff426f0e84..66406231a108d 100644 --- a/python/ray/tune/trial.py +++ b/python/ray/tune/trial.py @@ -3,7 +3,7 @@ from __future__ import print_function from collections import namedtuple -import cloudpickle +import ray.cloudpickle as cloudpickle import copy from datetime import datetime import logging diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index 3978905cdaa5e..d29fda43ca3d2 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -153,8 +153,7 @@ def restore(cls, logger.warning("".join([ "Attempting to resume experiment from {}. ".format( - metadata_checkpoint_dir), - "This feature is experimental, " + metadata_checkpoint_dir), "This feature is experimental, " "and may not work with all search algorithms. ", "This will ignore any new changes to specification." ])) From 0348eb4a01f9b4851d6ce422cd872bb7f7cd34ee Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Thu, 27 Dec 2018 10:47:26 +0800 Subject: [PATCH 163/172] test fix and move env logic --- python/ray/tune/test/cluster_tests.py | 16 +++++++++------- python/ray/tune/tune.py | 7 ++++++- 2 files changed, 15 insertions(+), 8 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 144697cdce653..2d614d6f707e5 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -347,7 +347,7 @@ def test_cluster_down_full(start_connected_cluster, tmpdir): cluster = _start_new_cluster() trials = tune.run_experiments(all_experiments, resume=True) - assert len(trials) == 2 + assert len(trials) == 4 assert all(t.status in [Trial.TERMINATED, Trial.ERROR] for t in trials) cluster.shutdown() @@ -380,7 +380,8 @@ def test_cluster_rllib_restore(start_connected_cluster, tmpdir): # The trainable returns every 0.5 seconds, so this should not miss # the checkpoint. for i in range(50): - if os.path.exists(os.path.join(dirpath, TrialRunner.CKPT_FILE_NAME)): + if os.path.exists(os.path.join( + dirpath, "experiment/" + TrialRunner.CKPT_FILE_NAME)): # Inspect the internal trialrunner runner = TrialRunner.restore(dirpath) trials = runner.get_trials() @@ -449,7 +450,8 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): # The trainable returns every 0.5 seconds, so this should not miss # the checkpoint. for i in range(50): - if os.path.exists(os.path.join(dirpath, TrialRunner.CKPT_FILE_NAME)): + if os.path.exists(os.path.join( + dirpath, "experiment/" + TrialRunner.CKPT_FILE_NAME)): # Inspect the internal trialrunner runner = TrialRunner.restore(dirpath) trials = runner.get_trials() @@ -466,7 +468,7 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): cluster = _start_new_cluster() Experiment._register_if_needed(_Fail) - # Inspect the internal trialrunner just in case + # Inspect the internal trialrunner runner = TrialRunner.restore(dirpath) trials = runner.get_trials() assert trials[0].last_result["training_iteration"] == 3 @@ -483,7 +485,7 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): }, resume=True, raise_on_failed_trial=False) - assert all([t.status == Trial.ERROR for t in trials2]) - assert set([t.trial_id for t in trials2]) == set( - [t.trial_id for t in trials]) + assert all(t.status == Trial.ERROR for t in trials2) + assert set(t.trial_id for t in trials2) == set( + t.trial_id for t in trials) cluster.shutdown() diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index 3af36cb62145f..00d4689f53858 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -116,11 +116,16 @@ def run_experiments(experiments, runner = None restore = False + # TUNE_RESUME_PROMPT_OFF is for testing purposes and defaults + # `resume=False.` + if os.environ.get("TUNE_RESUME_PROMPT_OFF"): + resume = resume or False + if os.path.exists( os.path.join(checkpoint_dir, TrialRunner.CKPT_FILE_NAME)): if resume: restore = True - elif resume is None and not os.environ.get("TUNE_RESUME_PROMPT_OFF"): + elif resume is None: msg = "Would you like to resume your experiment from '{}'?".format( checkpoint_dir) restore = click.confirm(msg, default=True) From fc6802bd14c566f639d60495326b1e9bbeb2bb80 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Wed, 26 Dec 2018 19:01:31 -0800 Subject: [PATCH 164/172] fix --- python/ray/tune/test/cluster_tests.py | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 2d614d6f707e5..9930ff42d2bf4 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -379,18 +379,20 @@ def test_cluster_rllib_restore(start_connected_cluster, tmpdir): # Wait until the right checkpoint is saved. # The trainable returns every 0.5 seconds, so this should not miss # the checkpoint. + metadata_checkpoint_dir = os.path.join(dirpath, "experiment") for i in range(50): if os.path.exists(os.path.join( - dirpath, "experiment/" + TrialRunner.CKPT_FILE_NAME)): + metadata_checkpoint_dir, TrialRunner.CKPT_FILE_NAME)): # Inspect the internal trialrunner - runner = TrialRunner.restore(dirpath) + runner = TrialRunner.restore(metadata_checkpoint_dir) trials = runner.get_trials() last_res = trials[0].last_result if last_res is not None and last_res["training_iteration"]: break time.sleep(0.2) - if not os.path.exists(os.path.join(dirpath, TrialRunner.CKPT_FILE_NAME)): + if not os.path.exists(os.path.join( + metadata_checkpoint_dir, TrialRunner.CKPT_FILE_NAME)): raise RuntimeError("Checkpoint file didn't appear.") ray.shutdown() @@ -449,18 +451,20 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): # Wait until the right checkpoint is saved. # The trainable returns every 0.5 seconds, so this should not miss # the checkpoint. + metadata_checkpoint_dir = os.path.join(dirpath, "experiment") for i in range(50): if os.path.exists(os.path.join( - dirpath, "experiment/" + TrialRunner.CKPT_FILE_NAME)): + metadata_checkpoint_dir, TrialRunner.CKPT_FILE_NAME)): # Inspect the internal trialrunner - runner = TrialRunner.restore(dirpath) + runner = TrialRunner.restore(metadata_checkpoint_dir) trials = runner.get_trials() last_res = trials[0].last_result if last_res is not None and last_res["training_iteration"] == 3: break time.sleep(0.2) - if not os.path.exists(os.path.join(dirpath, TrialRunner.CKPT_FILE_NAME)): + if not os.path.exists(os.path.join( + metadata_checkpoint_dir, TrialRunner.CKPT_FILE_NAME)): raise RuntimeError("Checkpoint file didn't appear.") ray.shutdown() @@ -469,7 +473,7 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): Experiment._register_if_needed(_Fail) # Inspect the internal trialrunner - runner = TrialRunner.restore(dirpath) + runner = TrialRunner.restore(metadata_checkpoint_dir) trials = runner.get_trials() assert trials[0].last_result["training_iteration"] == 3 assert trials[0].status == Trial.PENDING From d1f1c0b91402a2664fa5375a1af4bfe70cb76d6f Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Thu, 27 Dec 2018 11:08:49 +0800 Subject: [PATCH 165/172] fix tests --- python/ray/tune/test/cluster_tests.py | 24 +++++++++++++----------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index 9930ff42d2bf4..fc364294614ac 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -346,7 +346,8 @@ def test_cluster_down_full(start_connected_cluster, tmpdir): cluster.shutdown() cluster = _start_new_cluster() - trials = tune.run_experiments(all_experiments, resume=True) + trials = tune.run_experiments( + all_experiments, resume=True, raise_on_failed_trial=False) assert len(trials) == 4 assert all(t.status in [Trial.TERMINATED, Trial.ERROR] for t in trials) cluster.shutdown() @@ -381,8 +382,9 @@ def test_cluster_rllib_restore(start_connected_cluster, tmpdir): # the checkpoint. metadata_checkpoint_dir = os.path.join(dirpath, "experiment") for i in range(50): - if os.path.exists(os.path.join( - metadata_checkpoint_dir, TrialRunner.CKPT_FILE_NAME)): + if os.path.exists( + os.path.join(metadata_checkpoint_dir, + TrialRunner.CKPT_FILE_NAME)): # Inspect the internal trialrunner runner = TrialRunner.restore(metadata_checkpoint_dir) trials = runner.get_trials() @@ -391,8 +393,8 @@ def test_cluster_rllib_restore(start_connected_cluster, tmpdir): break time.sleep(0.2) - if not os.path.exists(os.path.join( - metadata_checkpoint_dir, TrialRunner.CKPT_FILE_NAME)): + if not os.path.exists( + os.path.join(metadata_checkpoint_dir, TrialRunner.CKPT_FILE_NAME)): raise RuntimeError("Checkpoint file didn't appear.") ray.shutdown() @@ -453,8 +455,9 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): # the checkpoint. metadata_checkpoint_dir = os.path.join(dirpath, "experiment") for i in range(50): - if os.path.exists(os.path.join( - metadata_checkpoint_dir, TrialRunner.CKPT_FILE_NAME)): + if os.path.exists( + os.path.join(metadata_checkpoint_dir, + TrialRunner.CKPT_FILE_NAME)): # Inspect the internal trialrunner runner = TrialRunner.restore(metadata_checkpoint_dir) trials = runner.get_trials() @@ -463,8 +466,8 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): break time.sleep(0.2) - if not os.path.exists(os.path.join( - metadata_checkpoint_dir, TrialRunner.CKPT_FILE_NAME)): + if not os.path.exists( + os.path.join(metadata_checkpoint_dir, TrialRunner.CKPT_FILE_NAME)): raise RuntimeError("Checkpoint file didn't appear.") ray.shutdown() @@ -490,6 +493,5 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): resume=True, raise_on_failed_trial=False) assert all(t.status == Trial.ERROR for t in trials2) - assert set(t.trial_id for t in trials2) == set( - t.trial_id for t in trials) + assert set(t.trial_id for t in trials2) == set(t.trial_id for t in trials) cluster.shutdown() From 7650e9f1a3e11afd9057e54ba9e38e8301b39bee Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Thu, 27 Dec 2018 20:14:58 +0900 Subject: [PATCH 166/172] Update train.py --- python/ray/rllib/train.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/python/ray/rllib/train.py b/python/ray/rllib/train.py index 39a06d0d02b0c..5e03dfa588e4b 100755 --- a/python/ray/rllib/train.py +++ b/python/ray/rllib/train.py @@ -71,6 +71,10 @@ def create_parser(parser_creator=None): default="default", type=str, help="Name of the subdirectory under `local_dir` to put results in.") + parser.add_argument( + "--resume", + action="store_true", + help="Whether to attempt to resume previous Tune experiments.") parser.add_argument( "--env", default=None, type=str, help="The gym environment to use.") parser.add_argument( @@ -139,7 +143,8 @@ def run(args, parser): run_experiments( experiments, scheduler=_make_scheduler(args), - queue_trials=args.queue_trials) + queue_trials=args.queue_trials, + resume=args.resume) if __name__ == "__main__": From 54af15ccdaf9521fa6d3f66b05faba3940f687f2 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Thu, 27 Dec 2018 20:58:34 +0800 Subject: [PATCH 167/172] Update python/ray/tune/tune.py Co-Authored-By: richardliaw --- python/ray/tune/tune.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index 00d4689f53858..6037c18ed4daa 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -126,7 +126,7 @@ def run_experiments(experiments, if resume: restore = True elif resume is None: - msg = "Would you like to resume your experiment from '{}'?".format( + msg = "Found incomplete experiment at {}. Would you like to resume it?".format( checkpoint_dir) restore = click.confirm(msg, default=True) else: From 8503e1716cd5883c5ffcaeb8cb042725a636cd82 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Thu, 27 Dec 2018 21:00:00 +0800 Subject: [PATCH 168/172] Update python/ray/tune/tune.py Co-Authored-By: richardliaw --- python/ray/tune/tune.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index 6037c18ed4daa..86fc5b0efc630 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -128,7 +128,12 @@ def run_experiments(experiments, elif resume is None: msg = "Found incomplete experiment at {}. Would you like to resume it?".format( checkpoint_dir) - restore = click.confirm(msg, default=True) + restore = click.confirm(msg, default=True) + if restore: + logger.info("Tip: to always resume, pass resume=True to run_experiments()" + else: + logger.info("Tip: to always start a new experiment, pass resume=False to run_experiments()" + else: logger.info( "Did not find checkpoint file in {}.".format(checkpoint_dir)) From 4294f42a65ed29be3a24ed70ed265b26d895c84d Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Fri, 28 Dec 2018 00:16:31 -0800 Subject: [PATCH 169/172] fix py2 test --- python/ray/tune/trainable.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/python/ray/tune/trainable.py b/python/ray/tune/trainable.py index dfcdf530f70c9..0b50b55216e4b 100644 --- a/python/ray/tune/trainable.py +++ b/python/ray/tune/trainable.py @@ -10,6 +10,7 @@ import logging import os import pickle +from six import string_types import shutil import tempfile import time @@ -220,7 +221,7 @@ def save(self, checkpoint_dir=None): os.makedirs(checkpoint_dir) checkpoint = self._save(checkpoint_dir) saved_as_dict = False - if isinstance(checkpoint, str): + if isinstance(checkpoint, string_types): if (not checkpoint.startswith(checkpoint_dir) or checkpoint == checkpoint_dir): raise ValueError( @@ -238,7 +239,9 @@ def save(self, checkpoint_dir=None): with open(checkpoint_path, "wb") as f: pickle.dump(checkpoint, f) else: - raise ValueError("Return value from `_save` must be dict or str.") + raise ValueError( + "`_save` must return a dict or string type: {}".format( + str(type(checkpoint)))) pickle.dump({ "experiment_id": self._experiment_id, "iteration": self._iteration, From b8da076627e9b43fa5729314667d90daf36809dd Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Fri, 28 Dec 2018 16:21:31 +0800 Subject: [PATCH 170/172] grammar --- python/ray/tune/trial_runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/tune/trial_runner.py b/python/ray/tune/trial_runner.py index d29fda43ca3d2..eddfbc488d8c0 100644 --- a/python/ray/tune/trial_runner.py +++ b/python/ray/tune/trial_runner.py @@ -155,7 +155,7 @@ def restore(cls, "Attempting to resume experiment from {}. ".format( metadata_checkpoint_dir), "This feature is experimental, " "and may not work with all search algorithms. ", - "This will ignore any new changes to specification." + "This will ignore any new changes to the specification." ])) from ray.tune.suggest import BasicVariantGenerator From c92313f638b1ed126de19f1f7fefe81ec39f50f0 Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Fri, 28 Dec 2018 16:41:27 +0800 Subject: [PATCH 171/172] fix --- python/ray/tune/tune.py | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index 86fc5b0efc630..bc9b8535c7a20 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -126,14 +126,17 @@ def run_experiments(experiments, if resume: restore = True elif resume is None: - msg = "Found incomplete experiment at {}. Would you like to resume it?".format( + msg = "Found incomplete experiment at {}. Would you like to resume it?".format( checkpoint_dir) - restore = click.confirm(msg, default=True) - if restore: - logger.info("Tip: to always resume, pass resume=True to run_experiments()" - else: - logger.info("Tip: to always start a new experiment, pass resume=False to run_experiments()" - + restore = click.confirm(msg, default=True) + if restore: + logger.info( + "Tip: to always resume, pass resume=True to run_experiments()" + ) + else: + logger.info( + "Tip: to always start a new experiment, pass resume=False to run_experiments()" + ) else: logger.info( "Did not find checkpoint file in {}.".format(checkpoint_dir)) From 577da9b0fb962f21760044874fb66fbed00ccd7d Mon Sep 17 00:00:00 2001 From: Richard Liaw Date: Fri, 28 Dec 2018 19:44:59 +0800 Subject: [PATCH 172/172] fix travis --- python/ray/tune/test/cluster_tests.py | 2 +- python/ray/tune/tune.py | 14 ++++++-------- test/multi_node_test.py | 1 + 3 files changed, 8 insertions(+), 9 deletions(-) diff --git a/python/ray/tune/test/cluster_tests.py b/python/ray/tune/test/cluster_tests.py index fc364294614ac..75b1a4c545fa7 100644 --- a/python/ray/tune/test/cluster_tests.py +++ b/python/ray/tune/test/cluster_tests.py @@ -493,5 +493,5 @@ def test_cluster_interrupt(start_connected_cluster, tmpdir): resume=True, raise_on_failed_trial=False) assert all(t.status == Trial.ERROR for t in trials2) - assert set(t.trial_id for t in trials2) == set(t.trial_id for t in trials) + assert {t.trial_id for t in trials2} == {t.trial_id for t in trials} cluster.shutdown() diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index bc9b8535c7a20..e65e10c7b402a 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -126,17 +126,15 @@ def run_experiments(experiments, if resume: restore = True elif resume is None: - msg = "Found incomplete experiment at {}. Would you like to resume it?".format( - checkpoint_dir) + msg = ("Found incomplete experiment at {}. " + "Would you like to resume it?".format(checkpoint_dir)) restore = click.confirm(msg, default=True) if restore: - logger.info( - "Tip: to always resume, pass resume=True to run_experiments()" - ) + logger.info("Tip: to always resume, " + "pass resume=True to run_experiments()") else: - logger.info( - "Tip: to always start a new experiment, pass resume=False to run_experiments()" - ) + logger.info("Tip: to always start a new experiment, " + "pass resume=False to run_experiments()") else: logger.info( "Did not find checkpoint file in {}.".format(checkpoint_dir)) diff --git a/test/multi_node_test.py b/test/multi_node_test.py index b25ea8295b314..e323751b25453 100644 --- a/test/multi_node_test.py +++ b/test/multi_node_test.py @@ -394,6 +394,7 @@ def train_func(config, reporter): # add a reporter arg time.sleep(0.1) reporter(timesteps_total=i, mean_accuracy=i+97) # report metrics +os.environ["TUNE_RESUME_PROMPT_OFF"] = "True" ray.init(redis_address="{}") ray.tune.register_trainable("train_func", train_func)