Skip to content

Commit

Permalink
[autoscaler] GCP node provider (#2061)
Browse files Browse the repository at this point in the history
* 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 9cf4840.

* 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 23a0066.

* Revert "Remove 'set -i' from updater ssh command"

This reverts commit 5fa034c.

* 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 b6744e4.

* Fix autoscaler/updater.py lint error, remove unused variable
  • Loading branch information
hartikainen authored and richardliaw committed May 31, 2018
1 parent 117107c commit 74dc14d
Show file tree
Hide file tree
Showing 13 changed files with 937 additions and 63 deletions.
29 changes: 20 additions & 9 deletions python/ray/autoscaler/autoscaler.py
Original file line number Diff line number Diff line change
Expand Up @@ -22,8 +22,9 @@
get_default_config
from ray.autoscaler.updater import NodeUpdaterProcess
from ray.autoscaler.docker import dockerize_if_needed
from ray.autoscaler.tags import TAG_RAY_LAUNCH_CONFIG, \
TAG_RAY_RUNTIME_CONFIG, TAG_RAY_NODE_STATUS, TAG_RAY_NODE_TYPE, TAG_NAME
from ray.autoscaler.tags import (TAG_RAY_LAUNCH_CONFIG, TAG_RAY_RUNTIME_CONFIG,
TAG_RAY_NODE_STATUS, TAG_RAY_NODE_TYPE,
TAG_RAY_NODE_NAME)
import ray.services as services

REQUIRED, OPTIONAL = True, False
Expand Down Expand Up @@ -58,6 +59,7 @@
"availability_zone": (str, OPTIONAL), # e.g. us-east-1a
"module": (str,
OPTIONAL), # module, if using external node provider
"project_id": (None, OPTIONAL), # gcp project id, if using gcp
},
REQUIRED),

Expand Down Expand Up @@ -244,6 +246,14 @@ def __init__(self,
self.last_update_time = 0.0
self.update_interval_s = update_interval_s

# Expand local file_mounts to allow ~ in the paths. This can't be done
# earlier when the config is written since we might be on different
# platform and the expansion would result in wrong path.
self.config["file_mounts"] = {
remote: os.path.expanduser(local)
for remote, local in self.config["file_mounts"].items()
}

for local_path in self.config["file_mounts"].values():
assert os.path.exists(local_path)

Expand All @@ -254,8 +264,8 @@ def update(self):
self.reload_config(errors_fatal=False)
self._update()
except Exception as e:
print("StandardAutoscaler: Error during autoscaling: {}",
traceback.format_exc())
print("StandardAutoscaler: Error during autoscaling: {}"
"".format(traceback.format_exc()))
self.num_failures += 1
if self.num_failures > self.max_failures:
print("*** StandardAutoscaler: Too many errors, abort. ***")
Expand Down Expand Up @@ -446,17 +456,18 @@ def launch_new_node(self, count):
num_before = len(self.workers())
self.provider.create_node(
self.config["worker_nodes"], {
TAG_NAME: "ray-{}-worker".format(self.config["cluster_name"]),
TAG_RAY_NODE_TYPE: "Worker",
TAG_RAY_NODE_STATUS: "Uninitialized",
TAG_RAY_NODE_NAME: "ray-{}-worker".format(
self.config["cluster_name"]),
TAG_RAY_NODE_TYPE: "worker",
TAG_RAY_NODE_STATUS: "uninitialized",
TAG_RAY_LAUNCH_CONFIG: self.launch_hash,
}, count)
if len(self.workers()) <= num_before:
print("Warning: Num nodes failed to increase after node creation")

def workers(self):
return self.provider.nodes(tag_filters={
TAG_RAY_NODE_TYPE: "Worker",
TAG_RAY_NODE_TYPE: "worker",
})

def debug_string(self, nodes=None):
Expand Down Expand Up @@ -565,7 +576,7 @@ def add_content_hashes(path):
with open(os.path.join(dirpath, name), "rb") as f:
hasher.update(f.read())
else:
with open(path, 'r') as f:
with open(os.path.expanduser(path), "r") as f:
hasher.update(f.read().encode("utf-8"))

hasher.update(json.dumps(sorted(file_mounts.items())).encode("utf-8"))
Expand Down
11 changes: 6 additions & 5 deletions python/ray/autoscaler/commands.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@
hash_launch_conf, fillout_defaults
from ray.autoscaler.node_provider import get_node_provider, NODE_PROVIDERS
from ray.autoscaler.tags import TAG_RAY_NODE_TYPE, TAG_RAY_LAUNCH_CONFIG, \
TAG_NAME
TAG_RAY_NODE_NAME
from ray.autoscaler.updater import NodeUpdaterProcess


Expand Down Expand Up @@ -57,7 +57,7 @@ def teardown_cluster(config_file, yes):

provider = get_node_provider(config["provider"], config["cluster_name"])
head_node_tags = {
TAG_RAY_NODE_TYPE: "Head",
TAG_RAY_NODE_TYPE: "head",
}
for node in provider.nodes(head_node_tags):
print("Terminating head node {}".format(node))
Expand All @@ -76,7 +76,7 @@ def get_or_create_head_node(config, no_restart, yes):

provider = get_node_provider(config["provider"], config["cluster_name"])
head_node_tags = {
TAG_RAY_NODE_TYPE: "Head",
TAG_RAY_NODE_TYPE: "head",
}
nodes = provider.nodes(head_node_tags)
if len(nodes) > 0:
Expand All @@ -98,7 +98,8 @@ def get_or_create_head_node(config, no_restart, yes):
provider.terminate_node(head_node)
print("Launching new head node...")
head_node_tags[TAG_RAY_LAUNCH_CONFIG] = launch_hash
head_node_tags[TAG_NAME] = "ray-{}-head".format(config["cluster_name"])
head_node_tags[TAG_RAY_NODE_NAME] = "ray-{}-head".format(
config["cluster_name"])
provider.create_node(config["head_node"], head_node_tags, 1)

nodes = provider.nodes(head_node_tags)
Expand Down Expand Up @@ -185,7 +186,7 @@ def get_head_node_ip(config_file):
config = yaml.load(open(config_file).read())
provider = get_node_provider(config["provider"], config["cluster_name"])
head_node_tags = {
TAG_RAY_NODE_TYPE: "Head",
TAG_RAY_NODE_TYPE: "head",
}
nodes = provider.nodes(head_node_tags)
if len(nodes) > 0:
Expand Down
Empty file.
Loading

0 comments on commit 74dc14d

Please sign in to comment.