ray/test/autoscaler_test.py

718 lines
25 KiB
Python
Raw Normal View History

from __future__ import absolute_import
from __future__ import division
from __future__ import print_function
import shutil
import tempfile
import threading
import time
import unittest
import yaml
import copy
import ray
import ray.services as services
from ray.autoscaler.autoscaler import StandardAutoscaler, LoadMetrics, \
fillout_defaults, validate_config
from ray.autoscaler.tags import TAG_RAY_NODE_TYPE, TAG_RAY_NODE_STATUS
from ray.autoscaler.node_provider import NODE_PROVIDERS, NodeProvider
from ray.autoscaler.updater import NodeUpdaterThread
class MockNode(object):
def __init__(self, node_id, tags):
self.node_id = node_id
self.state = "pending"
self.tags = tags
self.external_ip = "1.2.3.4"
self.internal_ip = "172.0.0.{}".format(self.node_id)
def matches(self, tags):
for k, v in tags.items():
if k not in self.tags or self.tags[k] != v:
return False
return True
class MockProcessRunner(object):
def __init__(self, fail_cmds=[]):
self.calls = []
self.fail_cmds = fail_cmds
def check_call(self, cmd, *args, **kwargs):
for token in self.fail_cmds:
if token in str(cmd):
raise Exception("Failing command on purpose")
self.calls.append(cmd)
class MockProvider(NodeProvider):
def __init__(self):
self.mock_nodes = {}
self.next_id = 0
self.throw = False
self.fail_creates = False
self.ready_to_create = threading.Event()
self.ready_to_create.set()
def nodes(self, tag_filters):
if self.throw:
raise Exception("oops")
return [
n.node_id for n in self.mock_nodes.values()
2018-04-11 10:11:35 -07:00
if n.matches(tag_filters) and n.state != "terminated"
]
def is_running(self, node_id):
return self.mock_nodes[node_id].state == "running"
def is_terminated(self, node_id):
return self.mock_nodes[node_id].state == "terminated"
def node_tags(self, node_id):
return self.mock_nodes[node_id].tags
def internal_ip(self, node_id):
return self.mock_nodes[node_id].internal_ip
def external_ip(self, node_id):
return self.mock_nodes[node_id].external_ip
def create_node(self, node_config, tags, count):
self.ready_to_create.wait()
if self.fail_creates:
return
for _ in range(count):
self.mock_nodes[self.next_id] = MockNode(self.next_id, tags)
self.next_id += 1
def set_node_tags(self, node_id, tags):
self.mock_nodes[node_id].tags.update(tags)
def terminate_node(self, node_id):
self.mock_nodes[node_id].state = "terminated"
SMALL_CLUSTER = {
"cluster_name": "default",
"min_workers": 2,
"max_workers": 2,
"target_utilization_fraction": 0.8,
"idle_timeout_minutes": 5,
"provider": {
"type": "mock",
"region": "us-east-1",
2018-01-10 13:18:34 -08:00
"availability_zone": "us-east-1a",
},
2018-02-20 00:24:01 -08:00
"docker": {
"image": "example",
"container_name": "mock",
},
"auth": {
"ssh_user": "ubuntu",
"ssh_private_key": "/dev/null",
},
"head_node": {
"TestProp": 1,
},
"worker_nodes": {
"TestProp": 2,
},
"file_mounts": {},
"setup_commands": ["cmd1"],
"head_setup_commands": ["cmd2"],
"worker_setup_commands": ["cmd3"],
"head_start_ray_commands": ["start_ray_head"],
"worker_start_ray_commands": ["start_ray_worker"],
}
class LoadMetricsTest(unittest.TestCase):
def testUpdate(self):
lm = LoadMetrics()
lm.update("1.1.1.1", {"CPU": 2}, {"CPU": 1})
self.assertEqual(lm.approx_workers_used(), 0.5)
lm.update("1.1.1.1", {"CPU": 2}, {"CPU": 0})
self.assertEqual(lm.approx_workers_used(), 1.0)
lm.update("2.2.2.2", {"CPU": 2}, {"CPU": 0})
self.assertEqual(lm.approx_workers_used(), 2.0)
def testPruneByNodeIp(self):
lm = LoadMetrics()
lm.update("1.1.1.1", {"CPU": 1}, {"CPU": 0})
lm.update("2.2.2.2", {"CPU": 1}, {"CPU": 0})
lm.prune_active_ips({"1.1.1.1", "4.4.4.4"})
self.assertEqual(lm.approx_workers_used(), 1.0)
def testBottleneckResource(self):
lm = LoadMetrics()
lm.update("1.1.1.1", {"CPU": 2}, {"CPU": 0})
lm.update("2.2.2.2", {"CPU": 2, "GPU": 16}, {"CPU": 2, "GPU": 2})
self.assertEqual(lm.approx_workers_used(), 1.88)
def testHeartbeat(self):
lm = LoadMetrics()
lm.update("1.1.1.1", {"CPU": 2}, {"CPU": 1})
lm.mark_active("2.2.2.2")
self.assertIn("1.1.1.1", lm.last_heartbeat_time_by_ip)
self.assertIn("2.2.2.2", lm.last_heartbeat_time_by_ip)
self.assertNotIn("3.3.3.3", lm.last_heartbeat_time_by_ip)
def testDebugString(self):
lm = LoadMetrics()
lm.update("1.1.1.1", {"CPU": 2}, {"CPU": 0})
lm.update("2.2.2.2", {"CPU": 2, "GPU": 16}, {"CPU": 2, "GPU": 2})
debug = lm.debug_string()
self.assertIn("ResourceUsage: 2.0/4.0 CPU, 14.0/16.0 GPU", debug)
self.assertIn("NumNodesConnected: 2", debug)
self.assertIn("NumNodesUsed: 1.88", debug)
class AutoscalingTest(unittest.TestCase):
def setUp(self):
NODE_PROVIDERS["mock"] = \
lambda: (None, self.create_provider)
self.provider = None
self.tmpdir = tempfile.mkdtemp()
def tearDown(self):
del NODE_PROVIDERS["mock"]
shutil.rmtree(self.tmpdir)
ray.worker.cleanup()
def waitFor(self, condition):
for _ in range(50):
if condition():
return
time.sleep(.1)
raise Exception("Timed out waiting for {}".format(condition))
def waitForNodes(self, expected, comparison=None, tag_filters={}):
MAX_ITER = 50
for i in range(MAX_ITER):
n = len(self.provider.nodes(tag_filters))
if comparison is None:
comparison = self.assertEqual
try:
comparison(n, expected)
return
except Exception:
if i == MAX_ITER - 1:
raise
time.sleep(.1)
def create_provider(self, config, cluster_name):
assert self.provider
return self.provider
def write_config(self, config):
path = self.tmpdir + "/simple.yaml"
with open(path, "w") as f:
f.write(yaml.dump(config))
return path
def testInvalidConfig(self):
invalid_config = "/dev/null"
self.assertRaises(
ValueError,
lambda: StandardAutoscaler(
invalid_config, LoadMetrics(), update_interval_s=0))
def testValidation(self):
"""Ensures that schema validation is working."""
config = copy.deepcopy(SMALL_CLUSTER)
try:
validate_config(config)
except Exception:
self.fail("Test config did not pass validation test!")
config["blah"] = "blah"
with self.assertRaises(ValueError):
validate_config(config)
del config["blah"]
config["provider"]["blah"] = "blah"
with self.assertRaises(ValueError):
validate_config(config)
del config["provider"]["blah"]
del config["provider"]
with self.assertRaises(ValueError):
validate_config(config)
def testValidateDefaultConfig(self):
config = {}
config["provider"] = {
"type": "aws",
"region": "us-east-1",
"availability_zone": "us-east-1a",
}
config = fillout_defaults(config)
try:
validate_config(config)
except Exception:
self.fail("Default config did not pass validation test!")
def testScaleUp(self):
config_path = self.write_config(SMALL_CLUSTER)
self.provider = MockProvider()
autoscaler = StandardAutoscaler(
config_path, LoadMetrics(), max_failures=0, update_interval_s=0)
self.assertEqual(len(self.provider.nodes({})), 0)
autoscaler.update()
self.waitForNodes(2)
autoscaler.update()
self.waitForNodes(2)
def testTerminateOutdatedNodesGracefully(self):
config = SMALL_CLUSTER.copy()
config["min_workers"] = 5
config["max_workers"] = 5
config_path = self.write_config(config)
self.provider = MockProvider()
[autoscaler] GCP node provider (#2061) * Google Cloud Platform scaffolding * Add minimal gcp config example * Add googleapiclient discoveries, update gcp.config constants * Rename and update gcp.config key pair name function * Implement gcp.config._configure_project * Fix the create project get project flow * Implement gcp.config._configure_iam_role * Implement service account iam binding * Implement gcp.config._configure_key_pair * Implement rsa key pair generation * Implement gcp.config._configure_subnet * Save work-in-progress gcp.config._configure_firewall_rules. These are likely to be not needed at all. Saving them if we happen to need them later. * Remove unnecessary firewall configuration * Update example-minimal.yaml configuration * Add new wait_for_compute_operation, rename old wait_for_operation * Temporarily rename autoscaler tags due to gcp incompatibility * Implement initial gcp.node_provider.nodes * Still missing filter support * Implement initial gcp.node_provider.create_node * Implement another compute wait operation (wait_For_compute_zone_operation). TODO: figure out if we can remove the function. * Implement initial gcp.node_provider._node and node status functions * Implement initial gcp.node_provider.terminate_node * Implement node tagging and ip getter methods for nodes * Temporarily rename tags due to gcp incompatibility * Tiny tweaks for autoscaler.updater * Remove unused config from gcp node_provider * Add new example-full example to gcp, update load_gcp_example_config * Implement label filtering for gcp.node_provider.nodes * Revert unnecessary change in ssh command * Revert "Temporarily rename tags due to gcp incompatibility" This reverts commit e2fe634c5d11d705c0f5d3e76c80c37394bb23fb. * Revert "Temporarily rename autoscaler tags due to gcp incompatibility" This reverts commit c938ee435f4b75854a14e78242ad7f1d1ed8ad4b. * Refactor autoscaler tagging to support multiple tag specs * Remove missing cryptography imports * Update quote function import * Fix threading issue in gcp.config with the compute discovery object * Add gcs support for log_sync * Fix the labels/tags naming discrepancy * Add expanduser to file_mounts hashing * Fix gcp.node_provider.internal_ip * Add uuid to node name * Remove 'set -i' from updater ssh command * Also add TODO with the context and reason for the change. * Update ssh key creation in autoscaler.gcp.config * Fix wait_for_compute_zone_operation's threading issue Google discovery api's compute object is not thread safe, and thus needs to be recreated for each thread. This moves the `wait_for_compute_zone_operation` under `autoscaler.gcp.config`, and adds compute as its argument. * Address pr feedback from @ericl * Expand local file mount paths in NodeUpdater * Add ssh_user name to key names * Update updater ssh to attempt 'set -i' and fall back if that fails * Update gcp/example-full.yaml * Fix wait crm operation in gcp.config * Update gcp/example-minimal.yaml to match aws/example-minimal.yaml * Fix gcp/example-full.yaml comment indentation * Add gcp/example-full.yaml to setup files * Update example-full.yaml command * Revert "Refactor autoscaler tagging to support multiple tag specs" This reverts commit 9cf48409ca2e5b66f800153853072c706fa502f6. * Update tag spec to only use characters [0-9a-z_-] * Change the tag values to conform gcp spec * Add project_id in the ssh key name * Replace '_' with '-' in autoscaler tag names * Revert "Update updater ssh to attempt 'set -i' and fall back if that fails" This reverts commit 23a0066c5254449e49746bd5e43b94b66f32bfb4. * Revert "Remove 'set -i' from updater ssh command" This reverts commit 5fa034cdf79fa7f8903691518c0d75699c630172. * Add fallback to `set -i` in force_interactive command * Update autoscaler tests to match current implementation * Update GCPNodeProvider.create_node to include hash in instance name * Add support for creating multiple instance on one create_node call * Clean TODOs * Update styles * Replace single quotes with double quotes * Some minor indentation fixes etc. * Remove unnecessary comment. Fix indentation. * Yapfify files that fail flake8 test * Yapfify more files * Update project_id handling in gcp node provider * temporary yapf mod * Revert "temporary yapf mod" This reverts commit b6744e4e15d4d936d1a14f4bf155ed1d3bb14126. * Fix autoscaler/updater.py lint error, remove unused variable
2018-05-31 09:00:03 -07:00
self.provider.create_node({}, {TAG_RAY_NODE_TYPE: "worker"}, 10)
autoscaler = StandardAutoscaler(
config_path, LoadMetrics(), max_failures=0, update_interval_s=0)
self.waitForNodes(10)
# Gradually scales down to meet target size, never going too low
for _ in range(10):
autoscaler.update()
self.waitForNodes(5, comparison=self.assertLessEqual)
self.waitForNodes(4, comparison=self.assertGreaterEqual)
# Eventually reaches steady state
self.waitForNodes(5)
def testDynamicScaling(self):
config_path = self.write_config(SMALL_CLUSTER)
self.provider = MockProvider()
autoscaler = StandardAutoscaler(
2018-04-11 10:11:35 -07:00
config_path,
LoadMetrics(),
max_launch_batch=5,
2018-04-11 10:11:35 -07:00
max_concurrent_launches=5,
max_failures=0,
update_interval_s=0)
self.waitForNodes(0)
autoscaler.update()
self.waitForNodes(2)
# Update the config to reduce the cluster size
new_config = SMALL_CLUSTER.copy()
new_config["max_workers"] = 1
self.write_config(new_config)
autoscaler.update()
self.waitForNodes(1)
# Update the config to reduce the cluster size
new_config["min_workers"] = 10
new_config["max_workers"] = 10
self.write_config(new_config)
autoscaler.update()
self.waitForNodes(6)
autoscaler.update()
self.waitForNodes(10)
def testDelayedLaunch(self):
config_path = self.write_config(SMALL_CLUSTER)
self.provider = MockProvider()
autoscaler = StandardAutoscaler(
config_path,
LoadMetrics(),
max_launch_batch=5,
max_concurrent_launches=5,
max_failures=0,
update_interval_s=0)
self.assertEqual(len(self.provider.nodes({})), 0)
# Update will try to create, but will block until we set the flag
self.provider.ready_to_create.clear()
autoscaler.update()
self.assertEqual(autoscaler.num_launches_pending.value, 2)
self.assertEqual(len(self.provider.nodes({})), 0)
# Set the flag, check it updates
self.provider.ready_to_create.set()
self.waitForNodes(2)
self.assertEqual(autoscaler.num_launches_pending.value, 0)
# Update the config to reduce the cluster size
new_config = SMALL_CLUSTER.copy()
new_config["max_workers"] = 1
self.write_config(new_config)
autoscaler.update()
self.assertEqual(len(self.provider.nodes({})), 1)
def testDelayedLaunchWithFailure(self):
config = SMALL_CLUSTER.copy()
config["min_workers"] = 10
config["max_workers"] = 10
config_path = self.write_config(config)
self.provider = MockProvider()
autoscaler = StandardAutoscaler(
config_path,
LoadMetrics(),
max_launch_batch=5,
max_concurrent_launches=8,
max_failures=0,
update_interval_s=0)
self.assertEqual(len(self.provider.nodes({})), 0)
# update() should launch a wave of 5 nodes (max_launch_batch)
# Force this first wave to block.
rtc1 = self.provider.ready_to_create
rtc1.clear()
autoscaler.update()
# Synchronization: wait for launchy thread to be blocked on rtc1
if hasattr(rtc1, '_cond'): # Python 3.5
waiters = rtc1._cond._waiters
else: # Python 2.7
waiters = rtc1._Event__cond._Condition__waiters
self.waitFor(lambda: len(waiters) == 1)
self.assertEqual(autoscaler.num_launches_pending.value, 5)
self.assertEqual(len(self.provider.nodes({})), 0)
# Call update() to launch a second wave of 3 nodes,
# as 5 + 3 = 8 = max_concurrent_launches.
# Make this wave complete immediately.
rtc2 = threading.Event()
self.provider.ready_to_create = rtc2
rtc2.set()
autoscaler.update()
self.waitForNodes(3)
self.assertEqual(autoscaler.num_launches_pending.value, 5)
# The first wave of 5 will now tragically fail
self.provider.fail_creates = True
rtc1.set()
self.waitFor(lambda: autoscaler.num_launches_pending.value == 0)
self.assertEqual(len(self.provider.nodes({})), 3)
# Retry the first wave, allowing it to succeed this time
self.provider.fail_creates = False
autoscaler.update()
self.waitForNodes(8)
self.assertEqual(autoscaler.num_launches_pending.value, 0)
# Final wave of 2 nodes
autoscaler.update()
self.waitForNodes(10)
self.assertEqual(autoscaler.num_launches_pending.value, 0)
def testUpdateThrottling(self):
config_path = self.write_config(SMALL_CLUSTER)
self.provider = MockProvider()
autoscaler = StandardAutoscaler(
2018-04-11 10:11:35 -07:00
config_path,
LoadMetrics(),
max_launch_batch=5,
2018-04-11 10:11:35 -07:00
max_concurrent_launches=5,
max_failures=0,
update_interval_s=10)
autoscaler.update()
self.waitForNodes(2)
self.assertEqual(autoscaler.num_launches_pending.value, 0)
new_config = SMALL_CLUSTER.copy()
new_config["max_workers"] = 1
self.write_config(new_config)
autoscaler.update()
# not updated yet
# note that node termination happens in the main thread, so
# we do not need to add any delay here before checking
self.assertEqual(len(self.provider.nodes({})), 2)
self.assertEqual(autoscaler.num_launches_pending.value, 0)
def testLaunchConfigChange(self):
config_path = self.write_config(SMALL_CLUSTER)
self.provider = MockProvider()
autoscaler = StandardAutoscaler(
config_path, LoadMetrics(), max_failures=0, update_interval_s=0)
autoscaler.update()
self.waitForNodes(2)
# Update the config to change the node type
new_config = SMALL_CLUSTER.copy()
new_config["worker_nodes"]["InstanceType"] = "updated"
self.write_config(new_config)
self.provider.ready_to_create.clear()
for _ in range(5):
autoscaler.update()
self.waitForNodes(0)
self.provider.ready_to_create.set()
self.waitForNodes(2)
def testIgnoresCorruptedConfig(self):
config_path = self.write_config(SMALL_CLUSTER)
self.provider = MockProvider()
autoscaler = StandardAutoscaler(
2018-04-11 10:11:35 -07:00
config_path,
LoadMetrics(),
max_launch_batch=10,
2018-04-11 10:11:35 -07:00
max_concurrent_launches=10,
max_failures=0,
update_interval_s=0)
autoscaler.update()
self.waitForNodes(2)
# Write a corrupted config
self.write_config("asdf")
for _ in range(10):
autoscaler.update()
time.sleep(0.1)
self.assertEqual(autoscaler.num_launches_pending.value, 0)
self.assertEqual(len(self.provider.nodes({})), 2)
# New a good config again
new_config = SMALL_CLUSTER.copy()
new_config["min_workers"] = 10
new_config["max_workers"] = 10
self.write_config(new_config)
autoscaler.update()
self.waitForNodes(10)
def testMaxFailures(self):
config_path = self.write_config(SMALL_CLUSTER)
self.provider = MockProvider()
self.provider.throw = True
autoscaler = StandardAutoscaler(
config_path, LoadMetrics(), max_failures=2, update_interval_s=0)
autoscaler.update()
autoscaler.update()
self.assertRaises(Exception, autoscaler.update)
def testLaunchNewNodeOnOutOfBandTerminate(self):
config_path = self.write_config(SMALL_CLUSTER)
self.provider = MockProvider()
autoscaler = StandardAutoscaler(
config_path, LoadMetrics(), max_failures=0, update_interval_s=0)
autoscaler.update()
autoscaler.update()
self.waitForNodes(2)
for node in self.provider.mock_nodes.values():
node.state = "terminated"
self.assertEqual(len(self.provider.nodes({})), 0)
autoscaler.update()
self.waitForNodes(2)
def testConfiguresNewNodes(self):
config_path = self.write_config(SMALL_CLUSTER)
self.provider = MockProvider()
runner = MockProcessRunner()
autoscaler = StandardAutoscaler(
2018-04-11 10:11:35 -07:00
config_path,
LoadMetrics(),
max_failures=0,
process_runner=runner,
verbose_updates=True,
node_updater_cls=NodeUpdaterThread,
update_interval_s=0)
autoscaler.update()
autoscaler.update()
self.waitForNodes(2)
for node in self.provider.mock_nodes.values():
node.state = "running"
2018-04-11 10:11:35 -07:00
assert len(
self.provider.nodes({
[autoscaler] GCP node provider (#2061) * Google Cloud Platform scaffolding * Add minimal gcp config example * Add googleapiclient discoveries, update gcp.config constants * Rename and update gcp.config key pair name function * Implement gcp.config._configure_project * Fix the create project get project flow * Implement gcp.config._configure_iam_role * Implement service account iam binding * Implement gcp.config._configure_key_pair * Implement rsa key pair generation * Implement gcp.config._configure_subnet * Save work-in-progress gcp.config._configure_firewall_rules. These are likely to be not needed at all. Saving them if we happen to need them later. * Remove unnecessary firewall configuration * Update example-minimal.yaml configuration * Add new wait_for_compute_operation, rename old wait_for_operation * Temporarily rename autoscaler tags due to gcp incompatibility * Implement initial gcp.node_provider.nodes * Still missing filter support * Implement initial gcp.node_provider.create_node * Implement another compute wait operation (wait_For_compute_zone_operation). TODO: figure out if we can remove the function. * Implement initial gcp.node_provider._node and node status functions * Implement initial gcp.node_provider.terminate_node * Implement node tagging and ip getter methods for nodes * Temporarily rename tags due to gcp incompatibility * Tiny tweaks for autoscaler.updater * Remove unused config from gcp node_provider * Add new example-full example to gcp, update load_gcp_example_config * Implement label filtering for gcp.node_provider.nodes * Revert unnecessary change in ssh command * Revert "Temporarily rename tags due to gcp incompatibility" This reverts commit e2fe634c5d11d705c0f5d3e76c80c37394bb23fb. * Revert "Temporarily rename autoscaler tags due to gcp incompatibility" This reverts commit c938ee435f4b75854a14e78242ad7f1d1ed8ad4b. * Refactor autoscaler tagging to support multiple tag specs * Remove missing cryptography imports * Update quote function import * Fix threading issue in gcp.config with the compute discovery object * Add gcs support for log_sync * Fix the labels/tags naming discrepancy * Add expanduser to file_mounts hashing * Fix gcp.node_provider.internal_ip * Add uuid to node name * Remove 'set -i' from updater ssh command * Also add TODO with the context and reason for the change. * Update ssh key creation in autoscaler.gcp.config * Fix wait_for_compute_zone_operation's threading issue Google discovery api's compute object is not thread safe, and thus needs to be recreated for each thread. This moves the `wait_for_compute_zone_operation` under `autoscaler.gcp.config`, and adds compute as its argument. * Address pr feedback from @ericl * Expand local file mount paths in NodeUpdater * Add ssh_user name to key names * Update updater ssh to attempt 'set -i' and fall back if that fails * Update gcp/example-full.yaml * Fix wait crm operation in gcp.config * Update gcp/example-minimal.yaml to match aws/example-minimal.yaml * Fix gcp/example-full.yaml comment indentation * Add gcp/example-full.yaml to setup files * Update example-full.yaml command * Revert "Refactor autoscaler tagging to support multiple tag specs" This reverts commit 9cf48409ca2e5b66f800153853072c706fa502f6. * Update tag spec to only use characters [0-9a-z_-] * Change the tag values to conform gcp spec * Add project_id in the ssh key name * Replace '_' with '-' in autoscaler tag names * Revert "Update updater ssh to attempt 'set -i' and fall back if that fails" This reverts commit 23a0066c5254449e49746bd5e43b94b66f32bfb4. * Revert "Remove 'set -i' from updater ssh command" This reverts commit 5fa034cdf79fa7f8903691518c0d75699c630172. * Add fallback to `set -i` in force_interactive command * Update autoscaler tests to match current implementation * Update GCPNodeProvider.create_node to include hash in instance name * Add support for creating multiple instance on one create_node call * Clean TODOs * Update styles * Replace single quotes with double quotes * Some minor indentation fixes etc. * Remove unnecessary comment. Fix indentation. * Yapfify files that fail flake8 test * Yapfify more files * Update project_id handling in gcp node provider * temporary yapf mod * Revert "temporary yapf mod" This reverts commit b6744e4e15d4d936d1a14f4bf155ed1d3bb14126. * Fix autoscaler/updater.py lint error, remove unused variable
2018-05-31 09:00:03 -07:00
TAG_RAY_NODE_STATUS: "uninitialized"
2018-04-11 10:11:35 -07:00
})) == 2
autoscaler.update()
self.waitForNodes(2, tag_filters={TAG_RAY_NODE_STATUS: "up-to-date"})
def testReportsConfigFailures(self):
config_path = self.write_config(SMALL_CLUSTER)
self.provider = MockProvider()
runner = MockProcessRunner(fail_cmds=["cmd1"])
autoscaler = StandardAutoscaler(
2018-04-11 10:11:35 -07:00
config_path,
LoadMetrics(),
max_failures=0,
process_runner=runner,
verbose_updates=True,
node_updater_cls=NodeUpdaterThread,
update_interval_s=0)
autoscaler.update()
autoscaler.update()
self.waitForNodes(2)
for node in self.provider.mock_nodes.values():
node.state = "running"
2018-04-11 10:11:35 -07:00
assert len(
self.provider.nodes({
[autoscaler] GCP node provider (#2061) * Google Cloud Platform scaffolding * Add minimal gcp config example * Add googleapiclient discoveries, update gcp.config constants * Rename and update gcp.config key pair name function * Implement gcp.config._configure_project * Fix the create project get project flow * Implement gcp.config._configure_iam_role * Implement service account iam binding * Implement gcp.config._configure_key_pair * Implement rsa key pair generation * Implement gcp.config._configure_subnet * Save work-in-progress gcp.config._configure_firewall_rules. These are likely to be not needed at all. Saving them if we happen to need them later. * Remove unnecessary firewall configuration * Update example-minimal.yaml configuration * Add new wait_for_compute_operation, rename old wait_for_operation * Temporarily rename autoscaler tags due to gcp incompatibility * Implement initial gcp.node_provider.nodes * Still missing filter support * Implement initial gcp.node_provider.create_node * Implement another compute wait operation (wait_For_compute_zone_operation). TODO: figure out if we can remove the function. * Implement initial gcp.node_provider._node and node status functions * Implement initial gcp.node_provider.terminate_node * Implement node tagging and ip getter methods for nodes * Temporarily rename tags due to gcp incompatibility * Tiny tweaks for autoscaler.updater * Remove unused config from gcp node_provider * Add new example-full example to gcp, update load_gcp_example_config * Implement label filtering for gcp.node_provider.nodes * Revert unnecessary change in ssh command * Revert "Temporarily rename tags due to gcp incompatibility" This reverts commit e2fe634c5d11d705c0f5d3e76c80c37394bb23fb. * Revert "Temporarily rename autoscaler tags due to gcp incompatibility" This reverts commit c938ee435f4b75854a14e78242ad7f1d1ed8ad4b. * Refactor autoscaler tagging to support multiple tag specs * Remove missing cryptography imports * Update quote function import * Fix threading issue in gcp.config with the compute discovery object * Add gcs support for log_sync * Fix the labels/tags naming discrepancy * Add expanduser to file_mounts hashing * Fix gcp.node_provider.internal_ip * Add uuid to node name * Remove 'set -i' from updater ssh command * Also add TODO with the context and reason for the change. * Update ssh key creation in autoscaler.gcp.config * Fix wait_for_compute_zone_operation's threading issue Google discovery api's compute object is not thread safe, and thus needs to be recreated for each thread. This moves the `wait_for_compute_zone_operation` under `autoscaler.gcp.config`, and adds compute as its argument. * Address pr feedback from @ericl * Expand local file mount paths in NodeUpdater * Add ssh_user name to key names * Update updater ssh to attempt 'set -i' and fall back if that fails * Update gcp/example-full.yaml * Fix wait crm operation in gcp.config * Update gcp/example-minimal.yaml to match aws/example-minimal.yaml * Fix gcp/example-full.yaml comment indentation * Add gcp/example-full.yaml to setup files * Update example-full.yaml command * Revert "Refactor autoscaler tagging to support multiple tag specs" This reverts commit 9cf48409ca2e5b66f800153853072c706fa502f6. * Update tag spec to only use characters [0-9a-z_-] * Change the tag values to conform gcp spec * Add project_id in the ssh key name * Replace '_' with '-' in autoscaler tag names * Revert "Update updater ssh to attempt 'set -i' and fall back if that fails" This reverts commit 23a0066c5254449e49746bd5e43b94b66f32bfb4. * Revert "Remove 'set -i' from updater ssh command" This reverts commit 5fa034cdf79fa7f8903691518c0d75699c630172. * Add fallback to `set -i` in force_interactive command * Update autoscaler tests to match current implementation * Update GCPNodeProvider.create_node to include hash in instance name * Add support for creating multiple instance on one create_node call * Clean TODOs * Update styles * Replace single quotes with double quotes * Some minor indentation fixes etc. * Remove unnecessary comment. Fix indentation. * Yapfify files that fail flake8 test * Yapfify more files * Update project_id handling in gcp node provider * temporary yapf mod * Revert "temporary yapf mod" This reverts commit b6744e4e15d4d936d1a14f4bf155ed1d3bb14126. * Fix autoscaler/updater.py lint error, remove unused variable
2018-05-31 09:00:03 -07:00
TAG_RAY_NODE_STATUS: "uninitialized"
2018-04-11 10:11:35 -07:00
})) == 2
autoscaler.update()
self.waitForNodes(
2, tag_filters={TAG_RAY_NODE_STATUS: "update-failed"})
def testConfiguresOutdatedNodes(self):
config_path = self.write_config(SMALL_CLUSTER)
self.provider = MockProvider()
runner = MockProcessRunner()
autoscaler = StandardAutoscaler(
2018-04-11 10:11:35 -07:00
config_path,
LoadMetrics(),
max_failures=0,
process_runner=runner,
verbose_updates=True,
node_updater_cls=NodeUpdaterThread,
update_interval_s=0)
autoscaler.update()
autoscaler.update()
self.waitForNodes(2)
for node in self.provider.mock_nodes.values():
node.state = "running"
autoscaler.update()
self.waitForNodes(2, tag_filters={TAG_RAY_NODE_STATUS: "up-to-date"})
runner.calls = []
new_config = SMALL_CLUSTER.copy()
new_config["worker_setup_commands"] = ["cmdX", "cmdY"]
self.write_config(new_config)
autoscaler.update()
autoscaler.update()
self.waitFor(lambda: len(runner.calls) > 0)
def testScaleUpBasedOnLoad(self):
config = SMALL_CLUSTER.copy()
config["min_workers"] = 1
config["max_workers"] = 10
config["target_utilization_fraction"] = 0.5
config_path = self.write_config(config)
self.provider = MockProvider()
lm = LoadMetrics()
autoscaler = StandardAutoscaler(
config_path, lm, max_failures=0, update_interval_s=0)
self.assertEqual(len(self.provider.nodes({})), 0)
autoscaler.update()
self.waitForNodes(1)
autoscaler.update()
self.assertEqual(autoscaler.num_launches_pending.value, 0)
self.assertEqual(len(self.provider.nodes({})), 1)
# Scales up as nodes are reported as used
local_ip = services.get_node_ip_address()
lm.update(local_ip, {"CPU": 2}, {"CPU": 0}) # head
lm.update("172.0.0.0", {"CPU": 2}, {"CPU": 0}) # worker 1
autoscaler.update()
self.waitForNodes(3)
lm.update("172.0.0.1", {"CPU": 2}, {"CPU": 0})
autoscaler.update()
self.waitForNodes(5)
# Holds steady when load is removed
lm.update("172.0.0.0", {"CPU": 2}, {"CPU": 2})
lm.update("172.0.0.1", {"CPU": 2}, {"CPU": 2})
autoscaler.update()
self.assertEqual(autoscaler.num_launches_pending.value, 0)
self.assertEqual(len(self.provider.nodes({})), 5)
# Scales down as nodes become unused
lm.last_used_time_by_ip["172.0.0.0"] = 0
lm.last_used_time_by_ip["172.0.0.1"] = 0
autoscaler.update()
self.assertEqual(autoscaler.num_launches_pending.value, 0)
self.assertEqual(len(self.provider.nodes({})), 3)
lm.last_used_time_by_ip["172.0.0.2"] = 0
lm.last_used_time_by_ip["172.0.0.3"] = 0
autoscaler.update()
self.assertEqual(autoscaler.num_launches_pending.value, 0)
self.assertEqual(len(self.provider.nodes({})), 1)
def testDontScaleBelowTarget(self):
config = SMALL_CLUSTER.copy()
config["min_workers"] = 0
config["max_workers"] = 2
config["target_utilization_fraction"] = 0.5
config_path = self.write_config(config)
self.provider = MockProvider()
lm = LoadMetrics()
autoscaler = StandardAutoscaler(
config_path, lm, max_failures=0, update_interval_s=0)
self.assertEqual(len(self.provider.nodes({})), 0)
autoscaler.update()
self.assertEqual(autoscaler.num_launches_pending.value, 0)
self.assertEqual(len(self.provider.nodes({})), 0)
# Scales up as nodes are reported as used
local_ip = services.get_node_ip_address()
lm.update(local_ip, {"CPU": 2}, {"CPU": 0}) # head
# 1.0 nodes used => target nodes = 2 => target workers = 1
autoscaler.update()
self.waitForNodes(1)
# Make new node idle, and never used.
# Should hold steady as target is still 2.
lm.update("172.0.0.0", {"CPU": 0}, {"CPU": 0})
lm.last_used_time_by_ip["172.0.0.0"] = 0
autoscaler.update()
self.assertEqual(len(self.provider.nodes({})), 1)
# Reduce load on head => target nodes = 1 => target workers = 0
lm.update(local_ip, {"CPU": 2}, {"CPU": 1})
autoscaler.update()
self.assertEqual(len(self.provider.nodes({})), 0)
def testRecoverUnhealthyWorkers(self):
config_path = self.write_config(SMALL_CLUSTER)
self.provider = MockProvider()
runner = MockProcessRunner()
lm = LoadMetrics()
autoscaler = StandardAutoscaler(
2018-04-11 10:11:35 -07:00
config_path,
lm,
max_failures=0,
process_runner=runner,
verbose_updates=True,
node_updater_cls=NodeUpdaterThread,
update_interval_s=0)
autoscaler.update()
self.waitForNodes(2)
for node in self.provider.mock_nodes.values():
node.state = "running"
autoscaler.update()
self.waitForNodes(2, tag_filters={TAG_RAY_NODE_STATUS: "up-to-date"})
# Mark a node as unhealthy
lm.last_heartbeat_time_by_ip["172.0.0.0"] = 0
num_calls = len(runner.calls)
autoscaler.update()
self.waitFor(lambda: len(runner.calls) > num_calls)
def testExternalNodeScaler(self):
config = SMALL_CLUSTER.copy()
config["provider"] = {
"type": "external",
"module": "ray.autoscaler.node_provider.NodeProvider",
2018-04-11 10:11:35 -07:00
}
config_path = self.write_config(config)
autoscaler = StandardAutoscaler(
config_path, LoadMetrics(), max_failures=0, update_interval_s=0)
self.assertIsInstance(autoscaler.provider, NodeProvider)
def testExternalNodeScalerWrongImport(self):
config = SMALL_CLUSTER.copy()
config["provider"] = {
"type": "external",
"module": "mymodule.provider_class",
2018-04-11 10:11:35 -07:00
}
invalid_provider = self.write_config(config)
self.assertRaises(
ImportError,
lambda: StandardAutoscaler(
invalid_provider, LoadMetrics(), update_interval_s=0))
def testExternalNodeScalerWrongModuleFormat(self):
config = SMALL_CLUSTER.copy()
config["provider"] = {
"type": "external",
"module": "does-not-exist",
2018-04-11 10:11:35 -07:00
}
invalid_provider = self.write_config(config)
self.assertRaises(
ValueError,
lambda: StandardAutoscaler(
invalid_provider, LoadMetrics(), update_interval_s=0))
if __name__ == "__main__":
unittest.main(verbosity=2)