From cf31c3f62254e7b725310ef6828729fa79a7742c Mon Sep 17 00:00:00 2001 From: Savin Date: Sat, 17 Jul 2021 11:03:32 -0700 Subject: [PATCH 01/32] Refactor @resources decorator @resources decorator is shared by all compute related decorators - @batch, @lambda, @k8s, @titus. This patch moves it out of batch_decorator.py so that other decorators can cleanly reference it. --- metaflow/plugins/__init__.py | 1 + metaflow/plugins/aws/batch/batch_decorator.py | 87 +++++++------------ metaflow/plugins/resources_decorator.py | 39 +++++++++ 3 files changed, 71 insertions(+), 56 deletions(-) create mode 100644 metaflow/plugins/resources_decorator.py diff --git a/metaflow/plugins/__init__.py b/metaflow/plugins/__init__.py index b386c2dd777..d6c6e9a29db 100644 --- a/metaflow/plugins/__init__.py +++ b/metaflow/plugins/__init__.py @@ -95,6 +95,7 @@ def _merge_lists(base, overrides, attr): from .timeout_decorator import TimeoutDecorator from .environment_decorator import EnvironmentDecorator from .retry_decorator import RetryDecorator +from .resources_decorator import ResourcesDecorator from .aws.batch.batch_decorator import BatchDecorator, ResourcesDecorator from .aws.step_functions.step_functions_decorator \ import StepFunctionsInternalDecorator diff --git a/metaflow/plugins/aws/batch/batch_decorator.py b/metaflow/plugins/aws/batch/batch_decorator.py index b1e9012377a..febfbf07bbb 100644 --- a/metaflow/plugins/aws/batch/batch_decorator.py +++ b/metaflow/plugins/aws/batch/batch_decorator.py @@ -10,6 +10,7 @@ from metaflow.datastore.util.s3util import get_s3_client from metaflow.decorators import StepDecorator from metaflow.metaflow_config import DATASTORE_LOCAL_DIR +from metaflow.plugins import ResourcesDecorator from metaflow.plugins.timeout_decorator import get_run_time_limit_for_task from metaflow.metadata import MetaDatum @@ -30,83 +31,57 @@ from urllib.parse import urlparse -class ResourcesDecorator(StepDecorator): - """ - Step decorator to specify the resources needed when executing this step. - This decorator passes this information along to Batch when requesting resources - to execute this step. - This decorator is ignored if the execution of the step does not happen on Batch. - To use, annotate your step as follows: - ``` - @resources(cpu=32) - @step - def myStep(self): - ... - ``` - Parameters - ---------- - cpu : int - Number of CPUs required for this step. Defaults to 1 - gpu : int - Number of GPUs required for this step. Defaults to 0 - memory : int - Memory size (in MB) required for this step. Defaults to 4096 - shared_memory : int - The value for the size (in MiB) of the /dev/shm volume for this step. - This parameter maps to the --shm-size option to docker run . - """ - name = 'resources' - defaults = { - 'cpu': '1', - 'gpu': '0', - 'memory': '4096', - 'shared_memory': None - } - class BatchDecorator(StepDecorator): """ - Step decorator to specify that this step should execute on Batch. - This decorator indicates that your step should execute on Batch. Note that you can - apply this decorator automatically to all steps using the ```--with batch``` argument - when calling run. Step level decorators are overrides and will force a step to execute - on Batch regardless of the ```--with``` specification. + Step decorator to specify that this step should execute on AWS Batch. + + This decorator indicates that your step should execute on AWS Batch. Note + that you can apply this decorator automatically to all steps using the + ```--with batch``` argument when calling run/resume. Step level decorators + within the code are overrides and will force a step to execute on AWS Batch + regardless of the ```--with``` specification. + To use, annotate your step as follows: ``` @batch @step - def myStep(self): + def my_step(self): ... ``` Parameters ---------- cpu : int - Number of CPUs required for this step. Defaults to 1. If @resources is also - present, the maximum value from all decorators is used + Number of CPUs required for this step. Defaults to 1. If @resources is + also present, the maximum value from all decorators is used gpu : int - Number of GPUs required for this step. Defaults to 0. If @resources is also - present, the maximum value from all decorators is used - memory : int - Memory size (in MB) required for this step. Defaults to 4096. If @resources is + Number of GPUs required for this step. Defaults to 0. If @resources is also present, the maximum value from all decorators is used + memory : int + Memory size (in MB) required for this step. Defaults to 4096. If + @resources is also present, the maximum value from all decorators is + used image : string - Image to use when launching on AWS Batch. If not specified, a default image mapping to - the current version of Python is used + Docker image to use when launching on AWS Batch. If not specified, a + default docker image mapping to the current version of Python is used queue : string - Queue to submit the job to. Defaults to the one determined by the environment variable - METAFLOW_BATCH_JOB_QUEUE + AWS Batch Job Queue to submit the job to. Defaults to the one + specified by the environment variable METAFLOW_BATCH_JOB_QUEUE iam_role : string - IAM role that AWS Batch can use to access Amazon S3. Defaults to the one determined by the environment - variable METAFLOW_ECS_S3_ACCESS_IAM_ROLE + AWS IAM role that AWS Batch container uses to access AWS cloud resources + (Amazon S3, Amazon DynamoDb, etc). Defaults to the one specified by the + environment variable METAFLOW_ECS_S3_ACCESS_IAM_ROLE execution_role : string - IAM role that AWS Batch can use to trigger AWS Fargate tasks. Defaults to the one determined by the environment - variable METAFLOW_ECS_FARGATE_EXECUTION_ROLE https://docs.aws.amazon.com/batch/latest/userguide/execution-IAM-role.html + AWS IAM role that AWS Batch can use to trigger AWS Fargate tasks. + Defaults to the one determined by the environment variable + METAFLOW_ECS_FARGATE_EXECUTION_ROLE https://docs.aws.amazon.com/batch/latest/userguide/execution-IAM-role.html shared_memory : int The value for the size (in MiB) of the /dev/shm volume for this step. This parameter maps to the --shm-size option to docker run. max_swap : int - The total amount of swap memory (in MiB) a container can use for this step. - This parameter is translated to the --memory-swap option to docker run - where the value is the sum of the container memory plus the max_swap value. + The total amount of swap memory (in MiB) a container can use for this + step. This parameter is translated to the --memory-swap option to + docker run where the value is the sum of the container memory plus the + max_swap value. swappiness : int This allows you to tune memory swappiness behavior for this step. A swappiness value of 0 causes swapping not to happen unless absolutely diff --git a/metaflow/plugins/resources_decorator.py b/metaflow/plugins/resources_decorator.py new file mode 100644 index 00000000000..38288a4ce66 --- /dev/null +++ b/metaflow/plugins/resources_decorator.py @@ -0,0 +1,39 @@ +from metaflow.decorators import StepDecorator + + +class ResourcesDecorator(StepDecorator): + """ + Step decorator to specify the resources needed when executing this step. + + This decorator passes this information along to container orchestrator + (AWS Batch, Kubernetes, etc.) when requesting resources to execute this + step. + + This decorator is ignored if the execution of the step happens locally. + + To use, annotate your step as follows: + ``` + @resources(cpu=32) + @step + def my_step(self): + ... + ``` + Parameters + ---------- + cpu : int + Number of CPUs required for this step. Defaults to 1 + gpu : int + Number of GPUs required for this step. Defaults to 0 + memory : int + Memory size (in MB) required for this step. Defaults to 4096 + shared_memory : int + The value for the size (in MiB) of the /dev/shm volume for this step. + This parameter maps to the --shm-size option to docker run . + """ + name = 'resources' + defaults = { + 'cpu': '1', + 'gpu': '0', + 'memory': '4096', + 'shared_memory': None + } \ No newline at end of file From c109bbb5c00c0de41e75b83cf3af96398010d82e Mon Sep 17 00:00:00 2001 From: Savin Date: Sat, 17 Jul 2021 11:46:40 -0700 Subject: [PATCH 02/32] Update __init__.py --- metaflow/plugins/__init__.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/metaflow/plugins/__init__.py b/metaflow/plugins/__init__.py index d6c6e9a29db..39face9e2b8 100644 --- a/metaflow/plugins/__init__.py +++ b/metaflow/plugins/__init__.py @@ -96,7 +96,7 @@ def _merge_lists(base, overrides, attr): from .environment_decorator import EnvironmentDecorator from .retry_decorator import RetryDecorator from .resources_decorator import ResourcesDecorator -from .aws.batch.batch_decorator import BatchDecorator, ResourcesDecorator +from .aws.batch.batch_decorator import BatchDecorator from .aws.step_functions.step_functions_decorator \ import StepFunctionsInternalDecorator from .test_unbounded_foreach_decorator\ From d93b98012b73f89c23040b9debb19a56b207bfd9 Mon Sep 17 00:00:00 2001 From: Savin Date: Sat, 17 Jul 2021 13:00:17 -0700 Subject: [PATCH 03/32] Refactor @batch decorator --- metaflow/plugins/aws/aws_utils.py | 108 +++++++ metaflow/plugins/aws/batch/batch_cli.py | 67 +---- metaflow/plugins/aws/batch/batch_client.py | 101 +------ metaflow/plugins/aws/batch/batch_decorator.py | 271 ++++++++---------- 4 files changed, 249 insertions(+), 298 deletions(-) create mode 100644 metaflow/plugins/aws/aws_utils.py diff --git a/metaflow/plugins/aws/aws_utils.py b/metaflow/plugins/aws/aws_utils.py new file mode 100644 index 00000000000..9f9a73c0f23 --- /dev/null +++ b/metaflow/plugins/aws/aws_utils.py @@ -0,0 +1,108 @@ +import os +import re +import tarfile + +from distutils.dir_util import copy_tree +try: + # python2 + from urlparse import urlparse +except: # noqa E722 + # python3 + from urllib.parse import urlparse + +from metaflow import util +from metaflow.datastore import MetaflowDataStore +from metaflow.datastore.local import LocalDataStore +from metaflow.datastore.util.s3util import get_s3_client + +def sync_metadata_to_S3(metadata_local_dir, datastore_root, retry_count): + with util.TempDir() as td: + tar_file_path = os.path.join(td, 'metadata.tgz') + with tarfile.open(tar_file_path, 'w:gz') as tar: + tar.add(metadata_local_dir) + # Upload metadata to Amazon S3. + with open(tar_file_path, 'rb') as f: + s3, _ = get_s3_client() + url = urlparse( + os.path.join( + datastore_root, + MetaflowDataStore.filename_with_attempt_prefix( + 'metadata.tgz', + retry_count))) + s3.upload_fileobj(f, url.netloc, url.path.lstrip('/')) + +def sync_metadata_from_S3(metadata_local_dir, datastore_root, retry_count): + def echo_none(*args, **kwargs): + pass + url = urlparse( + os.path.join( + datastore_root, + MetaflowDataStore.filename_with_attempt_prefix( + 'metadata.tgz', + retry_count))) + s3, err = get_s3_client() + try: + s3.head_object(Bucket=url.netloc, Key=url.path.lstrip('/')) + with util.TempDir() as td: + tar_file_path = os.path.join(td, 'metadata.tgz') + with open(tar_file_path, 'wb') as f: + s3.download_fileobj(url.netloc, url.path.lstrip('/'), f) + with tarfile.open(tar_file_path, 'r:gz') as tar: + tar.extractall(td) + copy_tree( + os.path.join(td, metadata_local_dir), + LocalDataStore.get_datastore_root_from_config(echo_none), + update=True) + except err as e: + # Metadata sync is best effort. + pass + +def get_docker_registry(image_uri): + """ + Explanation: + (.+?(?:[:.].+?)\/)? - [GROUP 0] REGISTRY + .+? - A registry must start with at least one character + (?:[:.].+?)\/ - A registry must have ":" or "." and end with "/" + ? - Make a registry optional + (.*?) - [GROUP 1] REPOSITORY + .*? - Get repository name until separator + (?:[@:])? - SEPARATOR + ?: - Don't capture separator + [@:] - The separator must be either "@" or ":" + ? - The separator is optional + ((?<=[@:]).*)? - [GROUP 2] TAG / DIGEST + (?<=[@:]) - A tag / digest must be preceeded by "@" or ":" + .* - Capture rest of tag / digest + ? - A tag / digest is optional + Examples: + image + - None + - image + - None + example/image + - None + - example/image + - None + example/image:tag + - None + - example/image + - tag + example.domain.com/example/image:tag + - example.domain.com/ + - example/image + - tag + 123.123.123.123:123/example/image:tag + - 123.123.123.123:123/ + - example/image + - tag + example.domain.com/example/image@sha256:45b23dee0 + - example.domain.com/ + - example/image + - sha256:45b23dee0 + """ + + pattern = re.compile(r"^(.+?(?:[:.].+?)\/)?(.*?)(?:[@:])?((?<=[@:]).*)?$") + registry, repository, tag = pattern.match(image_uri).groups() + if registry is not None: + registry = registry.rstrip("/") + return registry \ No newline at end of file diff --git a/metaflow/plugins/aws/batch/batch_cli.py b/metaflow/plugins/aws/batch/batch_cli.py index 0f80188a23b..1ab98beaa42 100644 --- a/metaflow/plugins/aws/batch/batch_cli.py +++ b/metaflow/plugins/aws/batch/batch_cli.py @@ -1,34 +1,19 @@ +import click import os import sys -import tarfile import time import traceback -import click - -from distutils.dir_util import copy_tree - -from .batch import Batch, BatchKilledException, STDOUT_PATH, STDERR_PATH - -from metaflow.datastore import MetaflowDataStore -from metaflow.datastore.local import LocalDataStore -from metaflow.datastore.util.s3util import get_s3_client -from metaflow.metaflow_config import DATASTORE_LOCAL_DIR from metaflow import util from metaflow import R -from metaflow.exception import ( - CommandException, - METAFLOW_EXIT_DISALLOW_RETRY, -) +from metaflow.exception import CommandException, METAFLOW_EXIT_DISALLOW_RETRY +from metaflow.metaflow_config import DATASTORE_LOCAL_DIR from metaflow.mflog import TASK_LOG_SOURCE +from .batch import Batch, BatchKilledException, STDOUT_PATH, STDERR_PATH + +from ..aws_utils import sync_metadata_from_S3 -try: - # python2 - from urlparse import urlparse -except: # noqa E722 - # python3 - from urllib.parse import urlparse @click.group() def cli(): @@ -63,34 +48,6 @@ def _execute_cmd(func, flow_name, run_id, user, my_runs, echo): func(flow_name, run_id, user, echo) -def _sync_metadata(echo, metadata, datastore_root, attempt): - if metadata.TYPE == 'local': - def echo_none(*args, **kwargs): - pass - path = os.path.join( - datastore_root, - MetaflowDataStore.filename_with_attempt_prefix('metadata.tgz', attempt)) - url = urlparse(path) - bucket = url.netloc - key = url.path.lstrip('/') - s3, err = get_s3_client() - try: - s3.head_object(Bucket=bucket, Key=key) - # If we are here, we can download the object - with util.TempDir() as td: - tar_file_path = os.path.join(td, 'metadata.tgz') - with open(tar_file_path, 'wb') as f: - s3.download_fileobj(bucket, key, f) - with tarfile.open(tar_file_path, 'r:gz') as tar: - tar.extractall(td) - copy_tree( - os.path.join(td, DATASTORE_LOCAL_DIR), - LocalDataStore.get_datastore_root_from_config(echo_none), - update=True) - except err as e: # noqa F841 - pass - - @batch.command(help="List unfinished AWS Batch tasks of this flow") @click.option("--my-runs", default=False, is_flag=True, help="List all my unfinished tasks.") @@ -272,6 +229,12 @@ def echo(msg, stream='stderr', batch_id=None): stdout_location = ds.get_log_location(TASK_LOG_SOURCE, 'stdout') stderr_location = ds.get_log_location(TASK_LOG_SOURCE, 'stderr') + def _sync_metadata(): + if ctx.obj.metadata.TYPE == 'local': + sync_metadata_from_S3(DATASTORE_LOCAL_DIR, + datastore_root, + retry_count) + batch = Batch(ctx.obj.metadata, ctx.obj.environment) try: with ctx.obj.monitor.measure("metaflow.batch.launch"): @@ -298,13 +261,13 @@ def echo(msg, stream='stderr', batch_id=None): ) except Exception as e: print(e) - _sync_metadata(echo, ctx.obj.metadata, datastore_root, retry_count) + _sync_metadata() sys.exit(METAFLOW_EXIT_DISALLOW_RETRY) try: batch.wait(stdout_location, stderr_location, echo=echo) except BatchKilledException: # don't retry killed tasks traceback.print_exc() - _sync_metadata(echo, ctx.obj.metadata, datastore_root, retry_count) + _sync_metadata() sys.exit(METAFLOW_EXIT_DISALLOW_RETRY) - _sync_metadata(echo, ctx.obj.metadata, datastore_root, retry_count) + _sync_metadata() diff --git a/metaflow/plugins/aws/batch/batch_client.py b/metaflow/plugins/aws/batch/batch_client.py index 192e982541f..fee8ca2e9fe 100644 --- a/metaflow/plugins/aws/batch/batch_client.py +++ b/metaflow/plugins/aws/batch/batch_client.py @@ -467,53 +467,6 @@ def wait_for_running(self): if not self.is_running and not self.is_done: BatchWaiter(self._client).wait_for_running(self.id) - @property - def log_stream_name(self): - return self.info['container'].get('logStreamName') - - def logs(self): - def get_log_stream(job): - log_stream_name = job.log_stream_name - if log_stream_name: - return BatchLogs('/aws/batch/job', log_stream_name, sleep_on_no_data=1) - else: - return None - - log_stream = None - while True: - if self.is_running or self.is_done or self.is_crashed: - log_stream = get_log_stream(self) - break - elif not self.is_done: - self.wait_for_running() - - if log_stream is None: - return - exception = None - for i in range(self.NUM_RETRIES + 1): - try: - check_after_done = 0 - for line in log_stream: - if not line: - if self.is_done: - if check_after_done > 1: - return - check_after_done += 1 - else: - pass - else: - i = 0 - yield line - return - except Exception as ex: - exception = ex - if self.is_crashed: - break - #sys.stderr.write(repr(ex) + '\n') - if i < self.NUM_RETRIES: - time.sleep(2 ** i + random.randint(0, 5)) - raise BatchJobException(repr(exception)) - def kill(self): if not self.is_done: self._client.terminate_job( @@ -570,56 +523,4 @@ def wait_for_running(self, job_id): ) self._waiter.create_waiter_with_client('JobRunning', model, self._client).wait( jobs=[job_id] - ) - -class BatchLogs(object): - def __init__(self, group, stream, pos=0, sleep_on_no_data=0): - from ..aws_client import get_aws_client - self._client = get_aws_client('logs') - self._group = group - self._stream = stream - self._pos = pos - self._sleep_on_no_data = sleep_on_no_data - self._buf = deque() - self._token = None - - def _get_events(self): - try: - if self._token: - response = self._client.get_log_events( - logGroupName=self._group, - logStreamName=self._stream, - startTime=self._pos, - nextToken=self._token, - startFromHead=True, - ) - else: - response = self._client.get_log_events( - logGroupName=self._group, - logStreamName=self._stream, - startTime=self._pos, - startFromHead=True, - ) - self._token = response['nextForwardToken'] - return response['events'] - except self._client.exceptions.ResourceNotFoundException as e: - # The logs might be delayed by a bit, so we can simply try - # again next time. - return [] - - def __iter__(self): - while True: - self._fill_buf() - if len(self._buf) == 0: - yield '' - if self._sleep_on_no_data > 0: - select.poll().poll(self._sleep_on_no_data * 1000) - else: - while self._buf: - yield self._buf.popleft() - - def _fill_buf(self): - events = self._get_events() - for event in events: - self._buf.append(event['message']) - self._pos = event['timestamp'] \ No newline at end of file + ) \ No newline at end of file diff --git a/metaflow/plugins/aws/batch/batch_decorator.py b/metaflow/plugins/aws/batch/batch_decorator.py index febfbf07bbb..ef57d20e013 100644 --- a/metaflow/plugins/aws/batch/batch_decorator.py +++ b/metaflow/plugins/aws/batch/batch_decorator.py @@ -1,35 +1,25 @@ import os import sys import platform -import re -import tarfile import requests -from metaflow.datastore import MetaflowDataStore -from metaflow.datastore.datastore import TransformableObject -from metaflow.datastore.util.s3util import get_s3_client +from metaflow import R +from metaflow import util from metaflow.decorators import StepDecorator -from metaflow.metaflow_config import DATASTORE_LOCAL_DIR +from metaflow.datastore.datastore import TransformableObject +from metaflow.metadata import MetaDatum +from metaflow.metaflow_config import ECS_S3_ACCESS_IAM_ROLE, \ + BATCH_JOB_QUEUE, \ + BATCH_CONTAINER_IMAGE, \ + BATCH_CONTAINER_REGISTRY, \ + ECS_FARGATE_EXECUTION_ROLE, \ + DATASTORE_LOCAL_DIR from metaflow.plugins import ResourcesDecorator from metaflow.plugins.timeout_decorator import get_run_time_limit_for_task -from metaflow.metadata import MetaDatum - -from metaflow import util -from metaflow import R - -from .batch import Batch, BatchException -from metaflow.metaflow_config import ECS_S3_ACCESS_IAM_ROLE, BATCH_JOB_QUEUE, \ - BATCH_CONTAINER_IMAGE, BATCH_CONTAINER_REGISTRY, \ - ECS_FARGATE_EXECUTION_ROLE from metaflow.sidecar import SidecarSubProcess -try: - # python2 - from urlparse import urlparse -except: # noqa E722 - # python3 - from urllib.parse import urlparse - +from .batch import BatchException +from ..aws_utils import get_docker_registry, sync_metadata_to_S3 class BatchDecorator(StepDecorator): """ @@ -108,40 +98,72 @@ def my_step(self): def __init__(self, attributes=None, statically_defined=False): super(BatchDecorator, self).__init__(attributes, statically_defined) + # If no docker image is explicitly specified, impute a default image. if not self.attributes['image']: + # If metaflow-config specifies a docker image, just use that. if BATCH_CONTAINER_IMAGE: self.attributes['image'] = BATCH_CONTAINER_IMAGE + # If metaflow-config doesn't specify a docker image, assign a + # default docker image. else: + # Metaflow-R has it's own default docker image (rocker family) if R.use_r(): self.attributes['image'] = R.container_image() + # Default to vanilla Python image corresponding to major.minor + # version of the Python interpreter launching the flow. else: - self.attributes['image'] = 'python:%s.%s' % (platform.python_version_tuple()[0], - platform.python_version_tuple()[1]) - if not BatchDecorator._get_registry(self.attributes['image']): + self.attributes['image'] = \ + 'python:%s.%s' % (platform.python_version_tuple()[0], + platform.python_version_tuple()[1]) + # Assign docker registry URL for the image. + if not get_docker_registry(self.attributes['image']): if BATCH_CONTAINER_REGISTRY: - self.attributes['image'] = '%s/%s' % (BATCH_CONTAINER_REGISTRY.rstrip('/'), - self.attributes['image']) + self.attributes['image'] = \ + '%s/%s' % (BATCH_CONTAINER_REGISTRY.rstrip('/'), + self.attributes['image']) - def step_init(self, flow, graph, step, decos, environment, datastore, logger): + # Refer https://github.com/Netflix/metaflow/blob/master/docs/lifecycle.png + # to understand where these functions are invoked in the lifecycle of a + # Metaflow flow. + def step_init(self, + flow, + graph, + step, + decos, + environment, + datastore, + logger): + # Executing AWS Batch jobs requires a non-local datastore. if datastore.TYPE != 's3': - raise BatchException('The *@batch* decorator requires --datastore=s3.') + raise KubernetesException( + 'The *@kubernetes* decorator requires --datastore=s3.') + # Set internal state. self.logger = logger self.environment = environment self.step = step for deco in decos: if isinstance(deco, ResourcesDecorator): for k, v in deco.attributes.items(): - # we use the larger of @resources and @batch attributes + # We use the larger of @resources and @k8s attributes + # TODO: Fix https://github.com/Netflix/metaflow/issues/467 my_val = self.attributes.get(k) if not (my_val is None and v is None): - self.attributes[k] = str(max(int(my_val or 0), int(v or 0))) + self.attributes[k] = \ + str(max(int(my_val or 0), int(v or 0))) + + # Set run time limit for the AWS Batch job. self.run_time_limit = get_run_time_limit_for_task(decos) if self.run_time_limit < 60: raise BatchException('The timeout for step *{step}* should be at ' - 'least 60 seconds for execution on AWS Batch'.format(step=step)) + 'least 60 seconds for execution on AWS Batch.'.format(step=step)) - def runtime_init(self, flow, graph, package, run_id): + def runtime_init(self, + flow, + graph, + package, + run_id): + # Set some more internal state. self.flow = flow self.graph = graph self.package = package @@ -154,8 +176,14 @@ def runtime_task_created(self, input_paths, is_cloned, ubf_context): - if not is_cloned: - self._save_package_once(datastore, self.package) + # To execute the AWS Batch job, the job container needs to have + # access to the code package. We store the package in the datastore + # which the pod is able to download as part of it's entrypoint. + if not is_cloned and self.package_url is None: + self.package_url = datastore.save_data( + self.package.sha, + TransformableObject(self.package.blob)) + self.package_sha = self.package.sha def runtime_step_cli(self, cli_args, @@ -164,7 +192,8 @@ def runtime_step_cli(self, ubf_context): if retry_count <= max_user_code_retries: # after all attempts to run the user code have failed, we don't need - # Batch anymore. We can execute possible fallback code locally. + # to execute on AWS Batch anymore. We can execute possible fallback + # code locally. cli_args.commands = ['batch', 'step'] cli_args.command_args.append(self.package_sha) cli_args.command_args.append(self.package_url) @@ -175,7 +204,7 @@ def runtime_step_cli(self, def task_pre_step(self, step_name, - ds, + datastore, metadata, run_id, task_id, @@ -184,119 +213,69 @@ def task_pre_step(self, retry_count, max_retries, ubf_context): - if metadata.TYPE == 'local': - self.ds_root = ds.root - else: - self.ds_root = None - meta = {} - meta['aws-batch-job-id'] = os.environ['AWS_BATCH_JOB_ID'] - meta['aws-batch-job-attempt'] = os.environ['AWS_BATCH_JOB_ATTEMPT'] - meta['aws-batch-ce-name'] = os.environ['AWS_BATCH_CE_NAME'] - meta['aws-batch-jq-name'] = os.environ['AWS_BATCH_JQ_NAME'] - meta['aws-batch-execution-env'] = os.environ['AWS_EXECUTION_ENV'] + # If `local` metadata is configured, we would need to copy task + # execution metadata from the AWS Batch container to user's + # local file system after the user code has finished execution. This + # happens via datastore as a communication bridge. + self.sync_metadata = metadata.TYPE == 'local' - # Capture AWS Logs metadata. This is best effort only since - # only V4 of the metadata uri for the ECS container hosts this - # information and it is quite likely that not all consumers of - # Metaflow would be running the container agent compatible with - # version V4. - # https://docs.aws.amazon.com/AmazonECS/latest/developerguide/task-metadata-endpoint.html - try: - logs_meta = requests.get( - url=os.environ['ECS_CONTAINER_METADATA_URI_V4']) \ - .json() \ - .get('LogOptions', {}) - meta['aws-batch-awslogs-group'] = logs_meta.get('awslogs-group') - meta['aws-batch-awslogs-region'] = logs_meta.get('awslogs-region') - meta['aws-batch-awslogs-stream'] = logs_meta.get('awslogs-stream') - except: - pass + self.datastore = datastore - entries = [MetaDatum(field=k, value=v, type=k, tags=[]) for k, v in meta.items()] - # Register book-keeping metadata for debugging. - metadata.register_metadata(run_id, step_name, task_id, entries) - self._save_logs_sidecar = SidecarSubProcess('save_logs_periodically') + # task_pre_step may run locally if fallback is activated for @catch + # decorator. In that scenario, we skip collecting AWS Batch execution + # metadata. A rudimentary way to detect non-local execution is to + # check for the existence of AWS_BATCH_JOB_ID environment variable. - def task_finished(self, step_name, flow, graph, is_task_ok, retry_count, max_retries): - if self.ds_root: - # We have a local metadata service so we need to persist it to the datastore. - # Note that the datastore is *always* s3 (see runtime_task_created function) - with util.TempDir() as td: - tar_file_path = os.path.join(td, 'metadata.tgz') - with tarfile.open(tar_file_path, 'w:gz') as tar: - # The local metadata is stored in the local datastore - # which, for batch jobs, is always the DATASTORE_LOCAL_DIR - tar.add(DATASTORE_LOCAL_DIR) - # At this point we upload what need to s3 - s3, _ = get_s3_client() - with open(tar_file_path, 'rb') as f: - path = os.path.join( - self.ds_root, - MetaflowDataStore.filename_with_attempt_prefix( - 'metadata.tgz', retry_count)) - url = urlparse(path) - s3.upload_fileobj(f, url.netloc, url.path.lstrip('/')) - try: - self._save_logs_sidecar.kill() - except: - pass + if 'AWS_BATCH_JOB_ID' in os.environ: + meta = {} + meta['aws-batch-job-id'] = os.environ['AWS_BATCH_JOB_ID'] + meta['aws-batch-job-attempt'] = os.environ['AWS_BATCH_JOB_ATTEMPT'] + meta['aws-batch-ce-name'] = os.environ['AWS_BATCH_CE_NAME'] + meta['aws-batch-jq-name'] = os.environ['AWS_BATCH_JQ_NAME'] + meta['aws-batch-execution-env'] = os.environ['AWS_EXECUTION_ENV'] - @classmethod - def _save_package_once(cls, datastore, package): - if cls.package_url is None: - cls.package_url = datastore.save_data(package.sha, TransformableObject(package.blob)) - cls.package_sha = package.sha + # Capture AWS Logs metadata. This is best effort only since + # only V4 of the metadata uri for the ECS container hosts this + # information and it is quite likely that not all consumers of + # Metaflow would be running the container agent compatible with + # version V4. + # https://docs.aws.amazon.com/AmazonECS/latest/developerguide/task-metadata-endpoint.html + try: + logs_meta = requests.get( + url=os.environ['ECS_CONTAINER_METADATA_URI_V4']) \ + .json() \ + .get('LogOptions', {}) + meta['aws-batch-awslogs-group'] = logs_meta.get('awslogs-group') + meta['aws-batch-awslogs-region'] = logs_meta.get('awslogs-region') + meta['aws-batch-awslogs-stream'] = logs_meta.get('awslogs-stream') + except: + pass - @classmethod - def _get_registry(cls, image): - """ - Explanation: + entries = \ + [MetaDatum(field=k, value=v, type=k, tags=[]) + for k, v in meta.items()] + # Register book-keeping metadata for debugging. + metadata.register_metadata(run_id, step_name, task_id, entries) + + # Start MFLog sidecar to collect task logs. + self._save_logs_sidecar = SidecarSubProcess('save_logs_periodically') - (.+?(?:[:.].+?)\/)? - [GROUP 0] REGISTRY - .+? - A registry must start with at least one character - (?:[:.].+?)\/ - A registry must have ":" or "." and end with "/" - ? - Make a registry optional - (.*?) - [GROUP 1] REPOSITORY - .*? - Get repository name until separator - (?:[@:])? - SEPARATOR - ?: - Don't capture separator - [@:] - The separator must be either "@" or ":" - ? - The separator is optional - ((?<=[@:]).*)? - [GROUP 2] TAG / DIGEST - (?<=[@:]) - A tag / digest must be preceeded by "@" or ":" - .* - Capture rest of tag / digest - ? - A tag / digest is optional - - Examples: - - image - - None - - image - - None - example/image - - None - - example/image - - None - example/image:tag - - None - - example/image - - tag - example.domain.com/example/image:tag - - example.domain.com/ - - example/image - - tag - 123.123.123.123:123/example/image:tag - - 123.123.123.123:123/ - - example/image - - tag - example.domain.com/example/image@sha256:45b23dee0 - - example.domain.com/ - - example/image - - sha256:45b23dee0 - """ + def task_finished(self, + step_name, + flow, + graph, + is_task_ok, + retry_count, + max_retries): + if self.sync_metadata: + # Note that the datastore is *always* Amazon S3 (see + # runtime_task_created function). + sync_metadata_to_S3(DATASTORE_LOCAL_DIR, + self.datastore.root, + retry_count) - pattern = re.compile(r"^(.+?(?:[:.].+?)\/)?(.*?)(?:[@:])?((?<=[@:]).*)?$") - registry, repository, tag = pattern.match(image).groups() - if registry is not None: - registry = registry.rstrip("/") - return registry + try: + self._save_logs_sidecar.kill() + except: + # Best effort kill + pass \ No newline at end of file From 89aa5d4cf839eea1c1009a109bdf10d25c6cc6c7 Mon Sep 17 00:00:00 2001 From: Savin Date: Sat, 17 Jul 2021 13:34:23 -0700 Subject: [PATCH 04/32] more change --- metaflow/plugins/aws/batch/batch_decorator.py | 36 ++++++++++--------- 1 file changed, 19 insertions(+), 17 deletions(-) diff --git a/metaflow/plugins/aws/batch/batch_decorator.py b/metaflow/plugins/aws/batch/batch_decorator.py index ef57d20e013..2dba069a157 100644 --- a/metaflow/plugins/aws/batch/batch_decorator.py +++ b/metaflow/plugins/aws/batch/batch_decorator.py @@ -213,12 +213,7 @@ def task_pre_step(self, retry_count, max_retries, ubf_context): - # If `local` metadata is configured, we would need to copy task - # execution metadata from the AWS Batch container to user's - # local file system after the user code has finished execution. This - # happens via datastore as a communication bridge. - self.sync_metadata = metadata.TYPE == 'local' - + self.metadata = metadata self.datastore = datastore # task_pre_step may run locally if fallback is activated for @catch @@ -267,15 +262,22 @@ def task_finished(self, is_task_ok, retry_count, max_retries): - if self.sync_metadata: - # Note that the datastore is *always* Amazon S3 (see - # runtime_task_created function). - sync_metadata_to_S3(DATASTORE_LOCAL_DIR, - self.datastore.root, - retry_count) + # task_finished may run locally if fallback is activated for @catch + # decorator. + if 'AWS_BATCH_JOB_ID' in os.environ: + # If `local` metadata is configured, we would need to copy task + # execution metadata from the AWS Batch container to user's + # local file system after the user code has finished execution. + # This happens via datastore as a communication bridge. + if self.metadata.TYPE == 'local': + # Note that the datastore is *always* Amazon S3 (see + # runtime_task_created function). + sync_metadata_to_S3(DATASTORE_LOCAL_DIR, + self.datastore.root, + retry_count) - try: - self._save_logs_sidecar.kill() - except: - # Best effort kill - pass \ No newline at end of file + try: + self._save_logs_sidecar.kill() + except: + # Best effort kill + pass \ No newline at end of file From 68123d364a8e2c68d611150fc20e31431f22ed3f Mon Sep 17 00:00:00 2001 From: Savin Date: Sat, 17 Jul 2021 13:46:32 -0700 Subject: [PATCH 05/32] more changes --- metaflow/plugins/aws/batch/batch.py | 2 -- metaflow/plugins/aws/batch/batch_cli.py | 2 +- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/metaflow/plugins/aws/batch/batch.py b/metaflow/plugins/aws/batch/batch.py index c2f91f3c0f6..128d90bf9d9 100644 --- a/metaflow/plugins/aws/batch/batch.py +++ b/metaflow/plugins/aws/batch/batch.py @@ -7,7 +7,6 @@ import time import warnings -from requests.exceptions import HTTPError from metaflow.exception import MetaflowException, MetaflowInternalError from metaflow.metaflow_config import BATCH_METADATA_SERVICE_URL, DATATOOLS_S3ROOT, \ DATASTORE_LOCAL_DIR, DATASTORE_SYSROOT_S3, DEFAULT_METADATA, \ @@ -231,7 +230,6 @@ def launch_job( cpu=None, gpu=None, memory=None, - platform=None, run_time_limit=None, shared_memory=None, max_swap=None, diff --git a/metaflow/plugins/aws/batch/batch_cli.py b/metaflow/plugins/aws/batch/batch_cli.py index 1ab98beaa42..0ded5bdaef4 100644 --- a/metaflow/plugins/aws/batch/batch_cli.py +++ b/metaflow/plugins/aws/batch/batch_cli.py @@ -237,7 +237,7 @@ def _sync_metadata(): batch = Batch(ctx.obj.metadata, ctx.obj.environment) try: - with ctx.obj.monitor.measure("metaflow.batch.launch"): + with ctx.obj.monitor.measure("metaflow.aws.batch.launch_job"): batch.launch_job( step_name, step_cli, From 4e6beccb51813438811c592e106b6b358a989d4e Mon Sep 17 00:00:00 2001 From: Savin Date: Sat, 17 Jul 2021 13:53:17 -0700 Subject: [PATCH 06/32] more changes --- metaflow/plugins/aws/batch/batch_decorator.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/metaflow/plugins/aws/batch/batch_decorator.py b/metaflow/plugins/aws/batch/batch_decorator.py index 2dba069a157..a545149513f 100644 --- a/metaflow/plugins/aws/batch/batch_decorator.py +++ b/metaflow/plugins/aws/batch/batch_decorator.py @@ -135,8 +135,8 @@ def step_init(self, logger): # Executing AWS Batch jobs requires a non-local datastore. if datastore.TYPE != 's3': - raise KubernetesException( - 'The *@kubernetes* decorator requires --datastore=s3.') + raise BatchException( + 'The *@batch* decorator requires --datastore=s3.') # Set internal state. self.logger = logger From f60b18ae231355d67e14541d15843903dc9f363e Mon Sep 17 00:00:00 2001 From: savin Date: Wed, 11 Aug 2021 08:47:47 -0700 Subject: [PATCH 07/32] @kubernetes --- metaflow/plugins/__init__.py | 4 + metaflow/plugins/aws/batch/batch.py | 365 ++++++++++-------- metaflow/plugins/aws/batch/batch_cli.py | 150 ++++--- metaflow/plugins/aws/eks/__init__.py | 0 metaflow/plugins/aws/eks/kubernetes.py | 364 +++++++++++++++++ metaflow/plugins/aws/eks/kubernetes_cli.py | 211 ++++++++++ metaflow/plugins/aws/eks/kubernetes_client.py | 176 +++++++++ .../plugins/aws/eks/kubernetes_decorator.py | 237 ++++++++++++ 8 files changed, 1291 insertions(+), 216 deletions(-) create mode 100644 metaflow/plugins/aws/eks/__init__.py create mode 100644 metaflow/plugins/aws/eks/kubernetes.py create mode 100644 metaflow/plugins/aws/eks/kubernetes_cli.py create mode 100644 metaflow/plugins/aws/eks/kubernetes_client.py create mode 100644 metaflow/plugins/aws/eks/kubernetes_decorator.py diff --git a/metaflow/plugins/__init__.py b/metaflow/plugins/__init__.py index 39face9e2b8..5918880cfd8 100644 --- a/metaflow/plugins/__init__.py +++ b/metaflow/plugins/__init__.py @@ -72,11 +72,13 @@ def get_plugin_cli(): # Add new CLI commands in this list from . import package_cli from .aws.batch import batch_cli + from .aws.eks import kubernetes_cli from .aws.step_functions import step_functions_cli return _ext_plugins.get_plugin_cli() + [ package_cli.cli, batch_cli.cli, + kubernetes_cli.cli, step_functions_cli.cli] @@ -97,6 +99,7 @@ def _merge_lists(base, overrides, attr): from .retry_decorator import RetryDecorator from .resources_decorator import ResourcesDecorator from .aws.batch.batch_decorator import BatchDecorator +from .aws.eks.kubernetes_decorator import KubernetesDecorator from .aws.step_functions.step_functions_decorator \ import StepFunctionsInternalDecorator from .test_unbounded_foreach_decorator\ @@ -109,6 +112,7 @@ def _merge_lists(base, overrides, attr): ResourcesDecorator, RetryDecorator, BatchDecorator, + KubernetesDecorator, StepFunctionsInternalDecorator, CondaStepDecorator, InternalTestUnboundedForeachDecorator], diff --git a/metaflow/plugins/aws/batch/batch.py b/metaflow/plugins/aws/batch/batch.py index 128d90bf9d9..c3bcea1b864 100644 --- a/metaflow/plugins/aws/batch/batch.py +++ b/metaflow/plugins/aws/batch/batch.py @@ -1,40 +1,46 @@ -import os -import time +import atexit import json +import os import select -import atexit import shlex import time import warnings -from metaflow.exception import MetaflowException, MetaflowInternalError -from metaflow.metaflow_config import BATCH_METADATA_SERVICE_URL, DATATOOLS_S3ROOT, \ - DATASTORE_LOCAL_DIR, DATASTORE_SYSROOT_S3, DEFAULT_METADATA, \ - BATCH_METADATA_SERVICE_HEADERS from metaflow import util - -from .batch_client import BatchClient - from metaflow.datastore.util.s3tail import S3Tail +from metaflow.exception import MetaflowException, MetaflowInternalError +from metaflow.metaflow_config import ( + BATCH_METADATA_SERVICE_URL, + DATATOOLS_S3ROOT, + DATASTORE_LOCAL_DIR, + DATASTORE_SYSROOT_S3, + DEFAULT_METADATA, + BATCH_METADATA_SERVICE_HEADERS, +) from metaflow.mflog.mflog import refine, set_should_persist -from metaflow.mflog import export_mflog_env_vars,\ - bash_capture_logs,\ - update_delay,\ - BASH_SAVE_LOGS +from metaflow.mflog import ( + export_mflog_env_vars, + bash_capture_logs, + update_delay, + BASH_SAVE_LOGS, +) + +from .batch_client import BatchClient # Redirect structured logs to /logs/ -LOGS_DIR = '/logs' -STDOUT_FILE = 'mflog_stdout' -STDERR_FILE = 'mflog_stderr' +LOGS_DIR = "/logs" +STDOUT_FILE = "mflog_stdout" +STDERR_FILE = "mflog_stderr" STDOUT_PATH = os.path.join(LOGS_DIR, STDOUT_FILE) STDERR_PATH = os.path.join(LOGS_DIR, STDERR_FILE) + class BatchException(MetaflowException): - headline = 'AWS Batch error' + headline = "AWS Batch error" class BatchKilledException(MetaflowException): - headline = 'AWS Batch task killed' + headline = "AWS Batch task killed" class Batch(object): @@ -42,22 +48,24 @@ def __init__(self, metadata, environment): self.metadata = metadata self.environment = environment self._client = BatchClient() - atexit.register(lambda: self.job.kill() if hasattr(self, 'job') else None) + atexit.register( + lambda: self.job.kill() if hasattr(self, "job") else None + ) - def _command(self, - environment, - code_package_url, - step_name, - step_cmds, - task_spec): - mflog_expr = export_mflog_env_vars(datastore_type='s3', - stdout_path=STDOUT_PATH, - stderr_path=STDERR_PATH, - **task_spec) + def _command( + self, environment, code_package_url, step_name, step_cmds, task_spec + ): + mflog_expr = export_mflog_env_vars( + datastore_type="s3", + stdout_path=STDOUT_PATH, + stderr_path=STDERR_PATH, + **task_spec + ) init_cmds = environment.get_package_commands(code_package_url) - init_expr = ' && '.join(init_cmds) - step_expr = bash_capture_logs(' && '.join( - environment.bootstrap_commands(step_name) + step_cmds)) + init_expr = " && ".join(init_cmds) + step_expr = bash_capture_logs( + " && ".join(environment.bootstrap_commands(step_name) + step_cmds) + ) # construct an entry point that # 1) initializes the mflog environment (mflog_expr) @@ -67,47 +75,52 @@ def _command(self, # the `true` command is to make sure that the generated command # plays well with docker containers which have entrypoint set as # eval $@ - cmd_str = 'true && mkdir -p /logs && %s && %s && %s; ' % \ - (mflog_expr, init_expr, step_expr) + cmd_str = "true && mkdir -p /logs && %s && %s && %s; " % ( + mflog_expr, + init_expr, + step_expr, + ) # after the task has finished, we save its exit code (fail/success) # and persist the final logs. The whole entrypoint should exit # with the exit code (c) of the task. # # Note that if step_expr OOMs, this tail expression is never executed. - # We lose the last logs in this scenario (although they are visible + # We lose the last logs in this scenario (although they are visible # still through AWS CloudWatch console). - cmd_str += 'c=$?; %s; exit $c' % BASH_SAVE_LOGS - return shlex.split('bash -c \"%s\"' % cmd_str) + cmd_str += "c=$?; %s; exit $c" % BASH_SAVE_LOGS + return shlex.split('bash -c "%s"' % cmd_str) def _search_jobs(self, flow_name, run_id, user): if user is None: - regex = '-{flow_name}-'.format(flow_name=flow_name) + regex = "-{flow_name}-".format(flow_name=flow_name) else: - regex = '{user}-{flow_name}-'.format( - user=user, flow_name=flow_name - ) + regex = "{user}-{flow_name}-".format(user=user, flow_name=flow_name) jobs = [] for job in self._client.unfinished_jobs(): - if regex in job['jobName']: - jobs.append(job['jobId']) + if regex in job["jobName"]: + jobs.append(job["jobId"]) if run_id is not None: - run_id = run_id[run_id.startswith('sfn-') and len('sfn-'):] + run_id = run_id[run_id.startswith("sfn-") and len("sfn-") :] for job in self._client.describe_jobs(jobs): - parameters = job['parameters'] - match = (user is None or parameters['metaflow.user'] == user) and \ - (parameters['metaflow.flow_name'] == flow_name) and \ - (run_id is None or parameters['metaflow.run_id'] == run_id) + parameters = job["parameters"] + match = ( + (user is None or parameters["metaflow.user"] == user) + and (parameters["metaflow.flow_name"] == flow_name) + and (run_id is None or parameters["metaflow.run_id"] == run_id) + ) if match: yield job - def _job_name(self, user, flow_name, run_id, step_name, task_id, retry_count): - return '{user}-{flow_name}-{run_id}-{step_name}-{task_id}-{retry_count}'.format( + def _job_name( + self, user, flow_name, run_id, step_name, task_id, retry_count + ): + return "{user}-{flow_name}-{run_id}-{step_name}-{task_id}-{retry_count}".format( user=user, flow_name=flow_name, - run_id=str(run_id) if run_id is not None else '', + run_id=str(run_id) if run_id is not None else "", step_name=step_name, - task_id=str(task_id) if task_id is not None else '', - retry_count=str(retry_count) if retry_count is not None else '' + task_id=str(task_id) if task_id is not None else "", + retry_count=str(retry_count) if retry_count is not None else "", ) def list_jobs(self, flow_name, run_id, user, echo): @@ -116,12 +129,12 @@ def list_jobs(self, flow_name, run_id, user, echo): for job in jobs: found = True echo( - '{name} [{id}] ({status})'.format( - name=job['jobName'], id=job['jobId'], status=job['status'] + "{name} [{id}] ({status})".format( + name=job["jobName"], id=job["jobId"], status=job["status"] ) ) if not found: - echo('No running AWS Batch jobs found.') + echo("No running AWS Batch jobs found.") def kill_jobs(self, flow_name, run_id, user, echo): jobs = self._search_jobs(flow_name, run_id, user) @@ -129,19 +142,21 @@ def kill_jobs(self, flow_name, run_id, user, echo): for job in jobs: found = True try: - self._client.attach_job(job['jobId']).kill() + self._client.attach_job(job["jobId"]).kill() echo( - 'Killing AWS Batch job: {name} [{id}] ({status})'.format( - name=job['jobName'], id=job['jobId'], status=job['status'] + "Killing AWS Batch job: {name} [{id}] ({status})".format( + name=job["jobName"], + id=job["jobId"], + status=job["status"], ) ) except Exception as e: echo( - 'Failed to terminate AWS Batch job %s [%s]' - % (job['jobId'], repr(e)) + "Failed to terminate AWS Batch job %s [%s]" + % (job["jobId"], repr(e)) ) if not found: - echo('No running AWS Batch jobs found.') + echo("No running AWS Batch jobs found.") def create_job( self, @@ -163,51 +178,77 @@ def create_job( max_swap=None, swappiness=None, env={}, - attrs={} + attrs={}, ): job_name = self._job_name( - attrs.get('metaflow.user'), - attrs.get('metaflow.flow_name'), - attrs.get('metaflow.run_id'), - attrs.get('metaflow.step_name'), - attrs.get('metaflow.task_id'), - attrs.get('metaflow.retry_count') + attrs.get("metaflow.user"), + attrs.get("metaflow.flow_name"), + attrs.get("metaflow.run_id"), + attrs.get("metaflow.step_name"), + attrs.get("metaflow.task_id"), + attrs.get("metaflow.retry_count"), ) job = self._client.job() - job \ - .job_name(job_name) \ - .job_queue(queue) \ - .command( - self._command(self.environment, code_package_url, - step_name, [step_cli], task_spec)) \ - .image(image) \ - .iam_role(iam_role) \ - .execution_role(execution_role) \ - .job_def(image, iam_role, - queue, execution_role, shared_memory, - max_swap, swappiness) \ - .cpu(cpu) \ - .gpu(gpu) \ - .memory(memory) \ - .shared_memory(shared_memory) \ - .max_swap(max_swap) \ - .swappiness(swappiness) \ - .timeout_in_secs(run_time_limit) \ - .environment_variable('AWS_DEFAULT_REGION', self._client.region()) \ - .environment_variable('METAFLOW_CODE_SHA', code_package_sha) \ - .environment_variable('METAFLOW_CODE_URL', code_package_url) \ - .environment_variable('METAFLOW_CODE_DS', code_package_ds) \ - .environment_variable('METAFLOW_USER', attrs['metaflow.user']) \ - .environment_variable('METAFLOW_SERVICE_URL', BATCH_METADATA_SERVICE_URL) \ - .environment_variable('METAFLOW_SERVICE_HEADERS', json.dumps(BATCH_METADATA_SERVICE_HEADERS)) \ - .environment_variable('METAFLOW_DATASTORE_SYSROOT_S3', DATASTORE_SYSROOT_S3) \ - .environment_variable('METAFLOW_DATATOOLS_S3ROOT', DATATOOLS_S3ROOT) \ - .environment_variable('METAFLOW_DEFAULT_DATASTORE', 's3') \ - .environment_variable('METAFLOW_DEFAULT_METADATA', DEFAULT_METADATA) - # Skip setting METAFLOW_DATASTORE_SYSROOT_LOCAL because metadata sync between the local user - # instance and the remote AWS Batch instance assumes metadata is stored in DATASTORE_LOCAL_DIR - # on the remote AWS Batch instance; this happens when METAFLOW_DATASTORE_SYSROOT_LOCAL - # is NOT set (see get_datastore_root_from_config in datastore/local.py). + job.job_name(job_name).job_queue(queue).command( + self._command( + self.environment, + code_package_url, + step_name, + [step_cli], + task_spec, + ) + ).image(image).iam_role(iam_role).execution_role( + execution_role + ).job_def( + image, + iam_role, + queue, + execution_role, + shared_memory, + max_swap, + swappiness, + ).cpu( + cpu + ).gpu( + gpu + ).memory( + memory + ).shared_memory( + shared_memory + ).max_swap( + max_swap + ).swappiness( + swappiness + ).timeout_in_secs( + run_time_limit + ).environment_variable( + "AWS_DEFAULT_REGION", self._client.region() + ).environment_variable( + "METAFLOW_CODE_SHA", code_package_sha + ).environment_variable( + "METAFLOW_CODE_URL", code_package_url + ).environment_variable( + "METAFLOW_CODE_DS", code_package_ds + ).environment_variable( + "METAFLOW_USER", attrs["metaflow.user"] + ).environment_variable( + "METAFLOW_SERVICE_URL", BATCH_METADATA_SERVICE_URL + ).environment_variable( + "METAFLOW_SERVICE_HEADERS", + json.dumps(BATCH_METADATA_SERVICE_HEADERS), + ).environment_variable( + "METAFLOW_DATASTORE_SYSROOT_S3", DATASTORE_SYSROOT_S3 + ).environment_variable( + "METAFLOW_DATATOOLS_S3ROOT", DATATOOLS_S3ROOT + ).environment_variable( + "METAFLOW_DEFAULT_DATASTORE", "s3" + ).environment_variable( + "METAFLOW_DEFAULT_METADATA", DEFAULT_METADATA + ) + # Skip setting METAFLOW_DATASTORE_SYSROOT_LOCAL because metadata sync between the local user + # instance and the remote AWS Batch instance assumes metadata is stored in DATASTORE_LOCAL_DIR + # on the remote AWS Batch instance; this happens when METAFLOW_DATASTORE_SYSROOT_LOCAL + # is NOT set (see get_datastore_root_from_config in datastore/local.py). for name, value in env.items(): job.environment_variable(name, value) if attrs: @@ -226,7 +267,7 @@ def launch_job( image, queue, iam_role=None, - execution_role=None, # for FARGATE compatibility + execution_role=None, # for FARGATE compatibility cpu=None, gpu=None, memory=None, @@ -236,60 +277,61 @@ def launch_job( swappiness=None, env={}, attrs={}, - ): + ): if queue is None: queue = next(self._client.active_job_queues(), None) if queue is None: raise BatchException( - 'Unable to launch AWS Batch job. No job queue ' - ' specified and no valid & enabled queue found.' + "Unable to launch AWS Batch job. No job queue " + " specified and no valid & enabled queue found." ) job = self.create_job( - step_name, - step_cli, - task_spec, - code_package_sha, - code_package_url, - code_package_ds, - image, - queue, - iam_role, - execution_role, - cpu, - gpu, - memory, - run_time_limit, - shared_memory, - max_swap, - swappiness, - env, - attrs + step_name, + step_cli, + task_spec, + code_package_sha, + code_package_url, + code_package_ds, + image, + queue, + iam_role, + execution_role, + cpu, + gpu, + memory, + run_time_limit, + shared_memory, + max_swap, + swappiness, + env, + attrs, ) self.job = job.execute() def wait(self, stdout_location, stderr_location, echo=None): - def wait_for_launch(job): status = job.status - echo('Task is starting (status %s)...' % status, - 'stderr', - batch_id=job.id) + echo( + "Task is starting (status %s)..." % status, + "stderr", + batch_id=job.id, + ) t = time.time() while True: - if status != job.status or (time.time()-t) > 30: + if status != job.status or (time.time() - t) > 30: status = job.status echo( - 'Task is starting (status %s)...' % status, - 'stderr', - batch_id=job.id + "Task is starting (status %s)..." % status, + "stderr", + batch_id=job.id, ) t = time.time() if job.is_running or job.is_done or job.is_crashed: break select.poll().poll(200) - prefix = b'[%s] ' % util.to_bytes(self.job.id) - + prefix = b"[%s] " % util.to_bytes(self.job.id) + def _print_available(tail, stream, should_persist=False): # print the latest batch of lines from S3Tail try: @@ -298,11 +340,14 @@ def _print_available(tail, stream, should_persist=False): line = set_should_persist(line) else: line = refine(line, prefix=prefix) - echo(line.strip().decode('utf-8', errors='replace'), stream) + echo(line.strip().decode("utf-8", errors="replace"), stream) except Exception as ex: - echo('[ temporary error in fetching logs: %s ]' % ex, - 'stderr', - batch_id=self.job.id) + echo( + "[ temporary error in fetching logs: %s ]" % ex, + "stderr", + batch_id=self.job.id, + ) + stdout_tail = S3Tail(stdout_location) stderr_tail = S3Tail(stderr_location) @@ -317,8 +362,8 @@ def _print_available(tail, stream, should_persist=False): while is_running: if time.time() > next_log_update: - _print_available(stdout_tail, 'stdout') - _print_available(stderr_tail, 'stderr') + _print_available(stdout_tail, "stdout") + _print_available(stderr_tail, "stderr") now = time.time() log_update_delay = update_delay(now - start_time) next_log_update = now + log_update_delay @@ -329,7 +374,7 @@ def _print_available(tail, stream, should_persist=False): # a long delay, regardless of the log tailing schedule d = min(log_update_delay, 5.0) select.poll().poll(d * 1000) - + # 3) Fetch remaining logs # # It is possible that we exit the loop above before all logs have been @@ -338,29 +383,33 @@ def _print_available(tail, stream, should_persist=False): # TODO if we notice AWS Batch failing to upload logs to S3, we can add a # HEAD request here to ensure that the file exists prior to calling # S3Tail and note the user about truncated logs if it doesn't - _print_available(stdout_tail, 'stdout') - _print_available(stderr_tail, 'stderr') + _print_available(stdout_tail, "stdout") + _print_available(stderr_tail, "stderr") # In case of hard crashes (OOM), the final save_logs won't happen. - # We fetch the remaining logs from AWS CloudWatch and persist them to + # We fetch the remaining logs from AWS CloudWatch and persist them to # Amazon S3. - # - # TODO: AWS CloudWatch fetch logs if self.job.is_crashed: - msg = next(msg for msg in - [self.job.reason, self.job.status_reason, 'Task crashed.'] - if msg is not None) + msg = next( + msg + for msg in [ + self.job.reason, + self.job.status_reason, + "Task crashed.", + ] + if msg is not None + ) raise BatchException( - '%s ' - 'This could be a transient error. ' - 'Use @retry to retry.' % msg + "%s " + "This could be a transient error. " + "Use @retry to retry." % msg ) else: if self.job.is_running: # Kill the job if it is still running by throwing an exception. raise BatchException("Task failed!") echo( - 'Task finished with exit code %s.' % self.job.status_code, - 'stderr', - batch_id=self.job.id + "Task finished with exit code %s." % self.job.status_code, + "stderr", + batch_id=self.job.id, ) diff --git a/metaflow/plugins/aws/batch/batch_cli.py b/metaflow/plugins/aws/batch/batch_cli.py index 0ded5bdaef4..e4d99cc5595 100644 --- a/metaflow/plugins/aws/batch/batch_cli.py +++ b/metaflow/plugins/aws/batch/batch_cli.py @@ -43,18 +43,28 @@ def _execute_cmd(func, flow_name, run_id, user, my_runs, echo): if not run_id and latest_run: run_id = util.get_latest_run_id(echo, flow_name) if run_id is None: - raise CommandException("A previous run id was not found. Specify --run-id.") + raise CommandException( + "A previous run id was not found. Specify --run-id." + ) func(flow_name, run_id, user, echo) @batch.command(help="List unfinished AWS Batch tasks of this flow") -@click.option("--my-runs", default=False, is_flag=True, - help="List all my unfinished tasks.") -@click.option("--user", default=None, - help="List unfinished tasks for the given user.") -@click.option("--run-id", default=None, - help="List unfinished tasks corresponding to the run id.") +@click.option( + "--my-runs", + default=False, + is_flag=True, + help="List all my unfinished tasks.", +) +@click.option( + "--user", default=None, help="List unfinished tasks for the given user." +) +@click.option( + "--run-id", + default=None, + help="List unfinished tasks corresponding to the run id.", +) @click.pass_context def list(ctx, run_id, user, my_runs): batch = Batch(ctx.obj.metadata, ctx.obj.environment) @@ -64,12 +74,22 @@ def list(ctx, run_id, user, my_runs): @batch.command(help="Terminate unfinished AWS Batch tasks of this flow.") -@click.option("--my-runs", default=False, is_flag=True, - help="Kill all my unfinished tasks.") -@click.option("--user", default=None, - help="Terminate unfinished tasks for the given user.") -@click.option("--run-id", default=None, - help="Terminate unfinished tasks corresponding to the run id.") +@click.option( + "--my-runs", + default=False, + is_flag=True, + help="Kill all my unfinished tasks.", +) +@click.option( + "--user", + default=None, + help="Terminate unfinished tasks for the given user.", +) +@click.option( + "--run-id", + default=None, + help="Terminate unfinished tasks corresponding to the run id.", +) @click.pass_context def kill(ctx, run_id, user, my_runs): batch = Batch(ctx.obj.metadata, ctx.obj.environment) @@ -79,24 +99,23 @@ def kill(ctx, run_id, user, my_runs): @batch.command( - help="Execute a single task using AWS Batch. This command " - "calls the top-level step command inside a AWS Batch " - "job with the given options. Typically you do not " - "call this command directly; it is used internally " - "by Metaflow." + help="Execute a single task using AWS Batch. This command calls the " + "top-level step command inside a AWS Batch job with the given options. " + "Typically you do not call this command directly; it is used internally by " + "Metaflow." ) @click.argument("step-name") @click.argument("code-package-sha") @click.argument("code-package-url") @click.option("--executable", help="Executable requirement for AWS Batch.") @click.option( - "--image", help="Docker image requirement for AWS Batch. In name:version format." + "--image", + help="Docker image requirement for AWS Batch. In name:version format.", ) +@click.option("--iam-role", help="IAM role requirement for AWS Batch.") @click.option( - "--iam-role", help="IAM role requirement for AWS Batch." -) -@click.option( - "--execution-role", help="Execution role requirement for AWS Batch on Fargate." + "--execution-role", + help="Execution role requirement for AWS Batch on Fargate.", ) @click.option("--cpu", help="CPU requirement for AWS Batch.") @click.option("--gpu", help="GPU requirement for AWS Batch.") @@ -111,21 +130,27 @@ def kill(ctx, run_id, user, my_runs): @click.option( "--tag", multiple=True, default=None, help="Passed to the top-level 'step'." ) -@click.option("--namespace", default=None, help="Passed to the top-level 'step'.") -@click.option("--retry-count", default=0, help="Passed to the top-level 'step'.") +@click.option( + "--namespace", default=None, help="Passed to the top-level 'step'." +) +@click.option( + "--retry-count", default=0, help="Passed to the top-level 'step'." +) @click.option( "--max-user-code-retries", default=0, help="Passed to the top-level 'step'." ) @click.option( "--run-time-limit", default=5 * 24 * 60 * 60, - help="Run time limit in seconds for the AWS Batch job. " "Default is 5 days." + help="Run time limit in seconds for the AWS Batch job. Default is 5 days.", +) +@click.option( + "--shared-memory", help="Shared Memory requirement for AWS Batch." ) -@click.option("--shared-memory", help="Shared Memory requirement for AWS Batch.") @click.option("--max-swap", help="Max Swap requirement for AWS Batch.") @click.option("--swappiness", help="Swappiness requirement for AWS Batch.") -#TODO: Maybe remove it altogether since it's not used here -@click.option('--ubf-context', default=None, type=click.Choice([None])) +# TODO: Maybe remove it altogether since it's not used here +@click.option("--ubf-context", default=None, type=click.Choice([None])) @click.pass_context def step( ctx, @@ -146,22 +171,23 @@ def step( swappiness=None, **kwargs ): - def echo(msg, stream='stderr', batch_id=None): + def echo(msg, stream="stderr", batch_id=None): msg = util.to_unicode(msg) if batch_id: - msg = '[%s] %s' % (batch_id, msg) + msg = "[%s] %s" % (batch_id, msg) ctx.obj.echo_always(msg, err=(stream == sys.stderr)) if ctx.obj.datastore.datastore_root is None: - ctx.obj.datastore.datastore_root = ctx.obj.datastore.get_datastore_root_from_config(echo) + ctx.obj.datastore.datastore_root = ( + ctx.obj.datastore.get_datastore_root_from_config(echo) + ) if R.use_r(): entrypoint = R.entrypoint() else: if executable is None: executable = ctx.obj.environment.executable(step_name) - entrypoint = '%s -u %s' % (executable, - os.path.basename(sys.argv[0])) + entrypoint = "%s -u %s" % (executable, os.path.basename(sys.argv[0])) top_args = " ".join(util.dict_to_cli_options(ctx.parent.parent.params)) @@ -170,14 +196,18 @@ def echo(msg, stream='stderr', batch_id=None): if input_paths: max_size = 30 * 1024 split_vars = { - "METAFLOW_INPUT_PATHS_%d" % (i // max_size): input_paths[i : i + max_size] + "METAFLOW_INPUT_PATHS_%d" + % (i // max_size): input_paths[i : i + max_size] for i in range(0, len(input_paths), max_size) } kwargs["input_paths"] = "".join("${%s}" % s for s in split_vars.keys()) step_args = " ".join(util.dict_to_cli_options(kwargs)) step_cli = u"{entrypoint} {top_args} step {step} {step_args}".format( - entrypoint=entrypoint, top_args=top_args, step=step_name, step_args=step_args + entrypoint=entrypoint, + top_args=top_args, + step=step_name, + step_args=step_args, ) node = ctx.obj.graph[step_name] @@ -192,17 +222,17 @@ def echo(msg, stream='stderr', batch_id=None): # Set batch attributes task_spec = { - 'flow_name': ctx.obj.flow.name, - 'step_name': step_name, - 'run_id': kwargs['run_id'], - 'task_id': kwargs['task_id'], - 'retry_count': str(retry_count) + "flow_name": ctx.obj.flow.name, + "step_name": step_name, + "run_id": kwargs["run_id"], + "task_id": kwargs["task_id"], + "retry_count": str(retry_count), } - attrs = {'metaflow.%s' % k: v for k, v in task_spec.items()} - attrs['metaflow.user'] = util.get_username() - attrs['metaflow.version'] = ctx.obj.environment.get_environment_info()[ - "metaflow_version" - ] + attrs = {"metaflow.%s" % k: v for k, v in task_spec.items()} + attrs["metaflow.user"] = util.get_username() + attrs["metaflow.version"] = ctx.obj.environment.get_environment_info()[ + "metaflow_version" + ] env_deco = [deco for deco in node.decorators if deco.name == "environment"] if env_deco: @@ -210,30 +240,34 @@ def echo(msg, stream='stderr', batch_id=None): else: env = {} - datastore_root = os.path.join(ctx.obj.datastore.make_path( - ctx.obj.flow.name, kwargs['run_id'], step_name, kwargs['task_id'])) + datastore_root = os.path.join( + ctx.obj.datastore.make_path( + ctx.obj.flow.name, kwargs["run_id"], step_name, kwargs["task_id"] + ) + ) # Add the environment variables related to the input-paths argument if split_vars: env.update(split_vars) if retry_count: ctx.obj.echo_always( - "Sleeping %d minutes before the next AWS Batch retry" % minutes_between_retries + "Sleeping %d minutes before the next AWS Batch retry" + % minutes_between_retries ) time.sleep(minutes_between_retries * 60) # this information is needed for log tailing spec = task_spec.copy() - spec['attempt'] = int(spec.pop('retry_count')) - ds = ctx.obj.datastore(mode='w', **spec) - stdout_location = ds.get_log_location(TASK_LOG_SOURCE, 'stdout') - stderr_location = ds.get_log_location(TASK_LOG_SOURCE, 'stderr') + spec["attempt"] = int(spec.pop("retry_count")) + ds = ctx.obj.datastore(mode="w", **spec) + stdout_location = ds.get_log_location(TASK_LOG_SOURCE, "stdout") + stderr_location = ds.get_log_location(TASK_LOG_SOURCE, "stderr") def _sync_metadata(): - if ctx.obj.metadata.TYPE == 'local': - sync_metadata_from_S3(DATASTORE_LOCAL_DIR, - datastore_root, - retry_count) + if ctx.obj.metadata.TYPE == "local": + sync_metadata_from_S3( + DATASTORE_LOCAL_DIR, datastore_root, retry_count + ) batch = Batch(ctx.obj.metadata, ctx.obj.environment) try: @@ -257,7 +291,7 @@ def _sync_metadata(): max_swap=max_swap, swappiness=swappiness, env=env, - attrs=attrs + attrs=attrs, ) except Exception as e: print(e) diff --git a/metaflow/plugins/aws/eks/__init__.py b/metaflow/plugins/aws/eks/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/metaflow/plugins/aws/eks/kubernetes.py b/metaflow/plugins/aws/eks/kubernetes.py new file mode 100644 index 00000000000..a65c60a411d --- /dev/null +++ b/metaflow/plugins/aws/eks/kubernetes.py @@ -0,0 +1,364 @@ +import os +import time +import json +import select +import atexit +import shlex +import time +import warnings + +from metaflow import util +from metaflow.datastore.util.s3tail import S3Tail +from metaflow.exception import MetaflowException, MetaflowInternalError +from metaflow.metaflow_config import ( + BATCH_METADATA_SERVICE_URL, + DATATOOLS_S3ROOT, + DATASTORE_LOCAL_DIR, + DATASTORE_SYSROOT_S3, + DEFAULT_METADATA, + BATCH_METADATA_SERVICE_HEADERS, +) +from metaflow.mflog import ( + export_mflog_env_vars, + bash_capture_logs, + update_delay, + BASH_SAVE_LOGS, +) +from metaflow.mflog.mflog import refine, set_should_persist + +from .kubernetes_client import KubernetesClient + +# Redirect structured logs to /logs/ +LOGS_DIR = "/logs" +STDOUT_FILE = "mflog_stdout" +STDERR_FILE = "mflog_stderr" +STDOUT_PATH = os.path.join(LOGS_DIR, STDOUT_FILE) +STDERR_PATH = os.path.join(LOGS_DIR, STDERR_FILE) + + +class KubernetesException(MetaflowException): + headline = "Kubernetes error" + + +class KubernetesKilledException(MetaflowException): + headline = "Kubernetes Batch job killed" + + +class Kubernetes(object): + def __init__( + self, + datastore, + metadata, + environment, + ): + self.datastore = datastore + self.metadata = metadata + self.environment = environment + + # TODO: Issue a kill request for all pending Kubernetes Batch jobs at exit. + # atexit.register( + # lambda: self.job.kill() if hasattr(self, 'job') else None) + + def _command( + self, + flow_name, + run_id, + step_name, + task_id, + attempt, + code_package_url, + step_cmds, + ): + print("lo") + mflog_expr = export_mflog_env_vars( + flow_name=flow_name, + run_id=run_id, + step_name=step_name, + task_id=task_id, + retry_count=attempt, + datastore_type=self.datastore.TYPE, + stdout_path=STDOUT_PATH, + stderr_path=STDERR_PATH, + ) + return ["echo", "hello"] + init_cmds = self.environment.get_package_commands(code_package_url) + init_expr = " && ".join(init_cmds) + step_expr = bash_capture_logs( + " && ".join( + self.environment.bootstrap_commands(step_name) + step_cmds + ) + ) + + # Construct an entry point that + # 1) initializes the mflog environment (mflog_expr) + # 2) bootstraps a metaflow environment (init_expr) + # 3) executes a task (step_expr) + + # The `true` command is to make sure that the generated command + # plays well with docker containers which have entrypoint set as + # eval $@ + cmd_str = "true && mkdir -p /logs && %s && %s && %s; " % ( + mflog_expr, + init_expr, + step_expr, + ) + # After the task has finished, we save its exit code (fail/success) + # and persist the final logs. The whole entrypoint should exit + # with the exit code (c) of the task. + # + # Note that if step_expr OOMs, this tail expression is never executed. + # We lose the last logs in this scenario. + # + # TODO: Find a way to capture hard exit logs. + cmd_str += "c=$?; %s; exit $c" % BASH_SAVE_LOGS + print("lo") + return shlex.split('bash -c "%s"' % cmd_str) + + def _name(self, user, flow_name, run_id, step_name, task_id, attempt): + print("hi") + return ( + "{user}-{flow_name}-{run_id}-" + "{step_name}-{task_id}-{attempt}".format( + user=user, + flow_name=flow_name, + run_id=str(run_id) if run_id is not None else "", + step_name=step_name, + task_id=str(task_id) if task_id is not None else "", + attempt=str(attempt) if attempt is not None else "", + ).lower() + ) + + def create_job( + self, + user, + flow_name, + run_id, + step_name, + task_id, + attempt, + code_package_sha, + code_package_url, + code_package_ds, + step_cli, + docker_image, + service_account=None, + cpu=None, + gpu=None, + memory=None, + run_time_limit=None, + env={}, + ): + job = ( + KubernetesClient() + .job() + .create( + name=self._name( + user=user, + flow_name=flow_name, + run_id=run_id, + step_name=step_name, + task_id=task_id, + attempt=attempt, + ) + ) + ) + + print(job) + print("lk") + # job.name("hello") + print("lk1") + print(job) + print("yahoo") + job.name( + self._name( + user=user, + flow_name=flow_name, + run_id=run_id, + step_name=step_name, + task_id=task_id, + attempt=attempt, + ) + ).command( + self._command( + flow_name=flow_name, + run_id=run_id, + step_name=step_name, + task_id=task_id, + attempt=attempt, + code_package_url=code_package_url, + step_cmds=[step_cli], + ) + ).image( + docker_image + ).service_account( + "s3-full-access" + ).cpu( + cpu + ).gpu( + gpu + ).memory( + memory + ).timeout_in_secs( + run_time_limit + ).environment_variable( + "AWS_DEFAULT_REGION", "us-west-2" + ).environment_variable( + "METAFLOW_CODE_SHA", code_package_sha + ).environment_variable( + "METAFLOW_CODE_URL", code_package_url + ).environment_variable( + "METAFLOW_CODE_DS", code_package_ds + ).environment_variable( + "METAFLOW_USER", user + ).environment_variable( + "METAFLOW_SERVICE_URL", BATCH_METADATA_SERVICE_URL + ).environment_variable( + "METAFLOW_SERVICE_HEADERS", + json.dumps(BATCH_METADATA_SERVICE_HEADERS), + ).environment_variable( + "METAFLOW_DATASTORE_SYSROOT_S3", DATASTORE_SYSROOT_S3 + ).environment_variable( + "METAFLOW_DATATOOLS_S3ROOT", DATATOOLS_S3ROOT + ).environment_variable( + "METAFLOW_DEFAULT_DATASTORE", "s3" + ).environment_variable( + "METAFLOW_DEFAULT_METADATA", DEFAULT_METADATA + ).annotation( + "metaflow-flow-name", flow_name + ).annotation( + "metaflow-run-id", run_id + ).annotation( + "metaflow-step-name", step_name + ).annotation( + "metaflow-task-id", task_id + ).annotation( + "metaflow-attempt", attempt + ).annotation( + "metaflow-user", user + ) + # Skip setting METAFLOW_DATASTORE_SYSROOT_LOCAL because metadata + # sync between the local user instance and the remote Kubernetes + # instance assumes metadata is stored in DATASTORE_LOCAL_DIR + # on the remote Kubernetes instance; this happens when + # METAFLOW_DATASTORE_SYSROOT_LOCAL is NOT set (see + # get_datastore_root_from_config in datastore/local.py). + for name, value in env.items(): + job.environment_variable(name, value) + + # A Container in a Pod may fail for a number of reasons, such as + # because the process in it exited with a non-zero exit code, or the + # Container was killed due to OOM etc. If this happens, fail the pod + # and let Metaflow handle the retries. + job.restart_policy("Never").retries(0) + + return job + + def wait(self, job, stdout_location, stderr_location, echo=None): + self.job = job + + def wait_for_launch(job): + status = job.status + echo( + "Task is starting (status %s)..." % status, + "stderr", + job_id=job.id, + ) + t = time.time() + while True: + if status != job.status or (time.time() - t) > 30: + status = job.status + echo( + "Task is starting (status %s)..." % status, + "stderr", + job_id=job.id, + ) + t = time.time() + if job.is_running or job.is_done or job.is_crashed: + break + select.poll().poll(200) + + prefix = b"[%s] " % util.to_bytes(self.job.id) + + def _print_available(tail, stream, should_persist=False): + # print the latest batch of lines from S3Tail + try: + for line in tail: + if should_persist: + line = set_should_persist(line) + else: + line = refine(line, prefix=prefix) + echo(line.strip().decode("utf-8", errors="replace"), stream) + except Exception as ex: + echo( + "[ temporary error in fetching logs: %s ]" % ex, + "stderr", + batch_id=self.job.id, + ) + + stdout_tail = S3Tail(stdout_location) + stderr_tail = S3Tail(stderr_location) + + # 1) Loop until the job has started + wait_for_launch(self.job) + + # 2) Loop until the job has finished + start_time = time.time() + is_running = True + next_log_update = start_time + log_update_delay = 1 + + while is_running: + if time.time() > next_log_update: + _print_available(stdout_tail, "stdout") + _print_available(stderr_tail, "stderr") + now = time.time() + log_update_delay = update_delay(now - start_time) + next_log_update = now + log_update_delay + is_running = self.job.is_running + + # This sleep should never delay log updates. On the other hand, + # we should exit this loop when the task has finished without + # a long delay, regardless of the log tailing schedule + d = min(log_update_delay, 5.0) + select.poll().poll(d * 1000) + + # 3) Fetch remaining logs + # + # It is possible that we exit the loop above before all logs have been + # shown. + # + # TODO if we notice AWS Batch failing to upload logs to S3, we can add a + # HEAD request here to ensure that the file exists prior to calling + # S3Tail and note the user about truncated logs if it doesn't + _print_available(stdout_tail, "stdout") + _print_available(stderr_tail, "stderr") + # In case of hard crashes (OOM), the final save_logs won't happen. + # We fetch the remaining logs from AWS CloudWatch and persist them to + # Amazon S3. + # + # TODO: AWS CloudWatch fetch logs + + if self.job.is_crashed: + msg = next( + msg + for msg in [ + self.job.reason, + self.job.status_reason, + "Task crashed.", + ] + if msg is not None + ) + raise BatchException( + "%s " + "This could be a transient error. " + "Use @retry to retry." % msg + ) + else: + if self.job.is_running: + # Kill the job if it is still running by throwing an exception. + raise BatchException("Task failed!") + echo( + "Task finished with exit code %s." % self.job.status_code, + "stderr", + batch_id=self.job.id, + ) diff --git a/metaflow/plugins/aws/eks/kubernetes_cli.py b/metaflow/plugins/aws/eks/kubernetes_cli.py new file mode 100644 index 00000000000..faffd7e9c6b --- /dev/null +++ b/metaflow/plugins/aws/eks/kubernetes_cli.py @@ -0,0 +1,211 @@ +import click +import os +import sys +import time +import traceback + +from metaflow import util +from metaflow.exception import CommandException, METAFLOW_EXIT_DISALLOW_RETRY + +from .kubernetes import Kubernetes, KubernetesKilledException +from ..aws_utils import sync_metadata_from_S3 + + +# TODO(s): +# 1. Compatibility for Metaflow-R (not a blocker for release). +# 2. + + +@click.group() +def cli(): + pass + + +@cli.group(help="Commands related to Kubernetes on Amazon EKS.") +def kubernetes(): + pass + + +@kubernetes.command( + help="Execute a single task on Kubernetes using Amazon EKS. This command " + "calls the top-level step command inside a Kubernetes job with the given " + "options. Typically you do not call this command directly; it is used " + "internally by Metaflow." +) +@click.argument("step-name") +@click.argument("code-package-sha") +@click.argument("code-package-url") +@click.option("--executable", help="Executable requirement for Kubernetes job.") +@click.option("--image", help="Docker image requirement for Kubernetes job.") +@click.option( + "--service-account", + # TODO: Support more auth mechanisms besides IRSA + help="IRSA requirement for Kubernetes job on Amazon EKS.", +) +@click.option("--cpu", help="CPU requirement for Kubernetes job.") +@click.option("--gpu", help="GPU requirement for Kubernetes job.") +@click.option("--memory", help="Memory requirement for Kubernetes job.") +@click.option("--run-id", help="Passed to the top-level 'step'.") +@click.option("--task-id", help="Passed to the top-level 'step'.") +@click.option("--input-paths", help="Passed to the top-level 'step'.") +@click.option("--split-index", help="Passed to the top-level 'step'.") +@click.option("--clone-path", help="Passed to the top-level 'step'.") +@click.option("--clone-run-id", help="Passed to the top-level 'step'.") +@click.option( + "--tag", multiple=True, default=None, help="Passed to the top-level 'step'." +) +@click.option( + "--namespace", default=None, help="Passed to the top-level 'step'." +) +@click.option( + "--retry-count", default=0, help="Passed to the top-level 'step'." +) +@click.option( + "--max-user-code-retries", default=0, help="Passed to the top-level 'step'." +) +@click.option( + "--run-time-limit", + default=5 * 24 * 60 * 60, # Default is set to 5 days + help="Run time limit in seconds for Kubernetes job.", +) +@click.pass_context +def step( + ctx, + step_name, + code_package_sha, + code_package_url, + executable=None, + image=None, + service_account=None, + cpu=None, + gpu=None, + memory=None, + run_time_limit=None, + **kwargs +): + def echo(msg, stream="stderr", job_id=None): + msg = util.to_unicode(msg) + if job_id: + msg = "[%s] %s" % (job_id, msg) + ctx.obj.echo_always(msg, err=(stream == sys.stderr)) + + node = ctx.obj.graph[step_name] + if ctx.obj.datastore.datastore_root is None: + ctx.obj.datastore.datastore_root = ( + ctx.obj.datastore.get_datastore_root_from_config(echo) + ) + + # Construct entrypoint CLI + if executable is None: + executable = ctx.obj.environment.executable(step_name) + entrypoint = "%s -u %s" % (executable, os.path.basename(sys.argv[0])) + + input_paths = kwargs.get("input_paths") + split_vars = None + if input_paths: + max_size = 30 * 1024 + split_vars = { + "METAFLOW_INPUT_PATHS_%d" + % (i // max_size): input_paths[i : i + max_size] + for i in range(0, len(input_paths), max_size) + } + kwargs["input_paths"] = "".join("${%s}" % s for s in split_vars.keys()) + + step_args = " ".join(util.dict_to_cli_options(kwargs)) + step_cli = u"{entrypoint} {top_args} step {step} {step_args}".format( + entrypoint=entrypoint, + top_args=" ".join(util.dict_to_cli_options(ctx.parent.parent.params)), + step=step_name, + step_args=" ".join(util.dict_to_cli_options(kwargs)), + ) + + # Set retry policy. + retry_count = kwargs.get("retry_count", 0) + retry_deco = [deco for deco in node.decorators if deco.name == "retry"] + minutes_between_retries = None + if retry_deco: + minutes_between_retries = int( + retry_deco[0].attributes.get("minutes_between_retries", 2) + ) + + # Set task attributes + task_spec = { + "flow_name": ctx.obj.flow.name, + "step_name": step_name, + "run_id": kwargs["run_id"], + "task_id": kwargs["task_id"], + "retry_count": str(retry_count), + } + attrs = {"metaflow.%s" % k: v for k, v in task_spec.items()} + attrs["metaflow.user"] = util.get_username() + attrs["metaflow.version"] = ctx.obj.environment.get_environment_info()[ + "metaflow_version" + ] + + # Set environment + env_deco = [deco for deco in node.decorators if deco.name == "environment"] + if env_deco: + env = env_deco[0].attributes["vars"] + else: + env = {} + + datastore_root = os.path.join( + ctx.obj.datastore.make_path( + ctx.obj.flow.name, kwargs["run_id"], step_name, kwargs["task_id"] + ) + ) + # Add the environment variables related to the input-paths argument + if split_vars: + env.update(split_vars) + + if retry_count: + ctx.obj.echo_always( + "Sleeping %d minutes before the next retry" + % minutes_between_retries + ) + time.sleep(minutes_between_retries * 60) + + # this information is needed for log tailing + spec = task_spec.copy() + spec["attempt"] = int(spec.pop("retry_count")) + # ds = ctx.obj.datastore(mode='w', **spec) + # stdout_location = ds.get_log_location(TASK_LOG_SOURCE, 'stdout') + # stderr_location = ds.get_log_location(TASK_LOG_SOURCE, 'stderr') + + kubernetes = Kubernetes( + ctx.obj.datastore, ctx.obj.metadata, ctx.obj.environment + ) + + try: + with ctx.obj.monitor.measure("metaflow.aws.eks.launch_job"): + job = kubernetes.create_job( + user=util.get_username(), + flow_name=ctx.obj.flow.name, + run_id=kwargs["run_id"], + step_name=step_name, + task_id=kwargs["task_id"], + attempt=str(retry_count), + code_package_sha=code_package_sha, + code_package_url=code_package_url, + code_package_ds=ctx.obj.datastore.TYPE, + step_cli=step_cli, + docker_image=image, + service_account=service_account, + cpu=cpu, + gpu=gpu, + memory=memory, + run_time_limit=run_time_limit, + env=env, + ).execute() + except Exception as e: + print(e) + sync_metadata_from_S3(ctx.obj.metadata, datastore_root, retry_count) + sys.exit(METAFLOW_EXIT_DISALLOW_RETRY) + try: + kubernetes.wait(job=job, echo=echo) + except KubernetesKilledException: + # don't retry killed tasks + traceback.print_exc() + sync_metadata_from_S3(ctx.obj.metadata, datastore_root, retry_count) + sys.exit(METAFLOW_EXIT_DISALLOW_RETRY) + sync_metadata_from_S3(ctx.obj.metadata, datastore_root, retry_count) diff --git a/metaflow/plugins/aws/eks/kubernetes_client.py b/metaflow/plugins/aws/eks/kubernetes_client.py new file mode 100644 index 00000000000..f640a7bd2d9 --- /dev/null +++ b/metaflow/plugins/aws/eks/kubernetes_client.py @@ -0,0 +1,176 @@ +from collections import defaultdict + +try: + unicode +except NameError: + unicode = str + basestring = str + +from metaflow.exception import MetaflowException + +# TODO (savin): Ensure that the client works swimmingly well with the sandbox. +class KubernetesClient(object): + def __init__(self): + # TODO (savin): Look into removing the usage of Kubernetes Python SDK + # at some point in the future. Given that Kubernetes Python SDK + # aggressively drops support for older kubernetes clusters, continued + # dependency on it may bite our users. + + # TODO (savin): Look into various ways to configure the Kubernetes + # client. + + # TODO (savin): Guard kubernetes import within a try..except. + try: + from kubernetes import client, config + except (NameError, ImportError): + raise MetaflowException( + "Could not import module 'kubernetes'. Install kubernetes Python package first." + ) + from kubernetes import client, config + + config.load_kube_config() + self._client = client.BatchV1Api() + + def job(self): + return KubernetesJob(self._client) + + def attach_job(self, job_id): + job = RunningJob(job_id, self._client) + return job.update() + + +class KubernetesJobException(MetaflowException): + headline = "Kubernetes job error" + + +class KubernetesJob(object): + def __init__(self, client): + # TODO (savin): Guard kubernetes import within a try..except. + # TODO(?) (savin): Remove dependency on Kubernetes Python SDK. + from kubernetes import client, config + + self._client = client + self._job = None + self._namespace = None + + def create( + self, + name, + namespace, + service_account, + annotations, + labels, + env, + command, + image, + cpu, + memory, + retries, + timeout_in_secs, + ): + self._namespace = namespace + self._job = client.V1Job( + api_version="batch/v1", + kind="Job", + metadata=client.V1ObjectMeta( + annotations=self._annotations, # annotations are for humans + labels=self._labels, # while labels are for kubernetes + name=self._name, # unique within the namespace + namespace=self._namespace, # empty defaults to `default` + ), + spec=client.V1JobSpec( + backoff_limit=self._retries, + ttl_seconds_after_finished=0, # delete the job immediately + template=client.V1PodTemplateSpec( + metadata=client.V1ObjectMeta( + annotations=self._annotations, + labels=self._labels, + name=self._name, + namespace=self._namespace, + ), + spec=client.V1PodSpec( + # Timeout is set on the pod + active_deadline_seconds=self._timeout_in_secs, + # affinity=?, + # automount_service_account_token=?, + containers=[ + client.V1Container( + command=self._command, + env=[ + client.V1EnvVar(name=k, value=v) + for k, v in self._env.items() + ], + image=self._image, + name=self._name, + resources=client.V1ResourceRequirements( + requests={ + "cpu": self._cpu, + "memory": self._memory, + } + ), + ) + ], + # image_pull_secrets=?, + # preemption_policy=?, + restart_policy="Never", + service_account_name=self._service_account, + # tolerations=?, + # volumes=?, + ), + ), + ), + ) + + def execute(self): + config.load_kube_config() + + response = client.BatchV1Api().create_namespaced_job( + body=self._job, namespace=self._namespace + ) + return RunningJob( + response.to_dict()["spec"]["template"]["metadata"]["name"], + self._client, + ) + + def name(self, name): + self._name = name + return self + + def command(self, command): + self._command = command + return self + + def image(self, image): + self._image = image + return self + + def cpu(self, cpu): + if not ( + isinstance(cpu, (int, unicode, basestring, float)) + and float(cpu) > 0 + ): + raise KubernetesJobException( + "Invalid CPU value ({}); it should be greater than 0".format( + cpu + ) + ) + self._cpu = str(cpu) + return self + + def memory(self, mem): + if not (isinstance(mem, (int, unicode, basestring)) and int(mem) > 0): + raise KubernetesJobException( + "Invalid memory value ({}); it should be greater than 0".format( + mem + ) + ) + self._memory = str(mem) + "M" + return self + + def environment_variable(self, name, value): + if name in self.env: + raise KubernetesJobException( + "Duplicate environment variable ({})".format(name) + ) + self._env[name] = str(value) + return self diff --git a/metaflow/plugins/aws/eks/kubernetes_decorator.py b/metaflow/plugins/aws/eks/kubernetes_decorator.py new file mode 100644 index 00000000000..0c3cb2aa91b --- /dev/null +++ b/metaflow/plugins/aws/eks/kubernetes_decorator.py @@ -0,0 +1,237 @@ +import os +import sys +import platform +import requests + +from metaflow import R +from metaflow import util +from metaflow.decorators import StepDecorator +from metaflow.datastore.datastore import TransformableObject +from metaflow.metadata import MetaDatum +from metaflow.metaflow_config import ( + ECS_S3_ACCESS_IAM_ROLE, + BATCH_JOB_QUEUE, + BATCH_CONTAINER_IMAGE, + BATCH_CONTAINER_REGISTRY, + ECS_FARGATE_EXECUTION_ROLE, + DATASTORE_LOCAL_DIR, +) +from metaflow.plugins import ResourcesDecorator +from metaflow.plugins.timeout_decorator import get_run_time_limit_for_task +from metaflow.sidecar import SidecarSubProcess + +from .kubernetes import KubernetesException +from ..aws_utils import get_docker_registry, sync_metadata_to_S3 + + +class KubernetesDecorator(StepDecorator): + """ + Step decorator to specify that this step should execute on Kubernetes. + + This decorator indicates that your step should execute on Kubernetes. Note + that you can apply this decorator automatically to all steps using the + ```--with kubernetes``` argument when calling run/resume. Step level + decorators within the code are overrides and will force a step to execute + on Kubernetes regardless of the ```--with``` specification. + + To use, annotate your step as follows: + ``` + @kubernetes + @step + def my_step(self): + ... + ``` + Parameters + ---------- + cpu : int + Number of CPUs required for this step. Defaults to 1. If @resources is + also present, the maximum value from all decorators is used + gpu : int + Number of GPUs required for this step. Defaults to 0. If @resources is + also present, the maximum value from all decorators is used + memory : int + Memory size (in MB) required for this step. Defaults to 4096. If + @resources is also present, the maximum value from all decorators is + used + image : string + Docker image to use when launching on Kubernetes. If not specified, a + default docker image mapping to the current version of Python is used + shared_memory : int + The value for the size (in MiB) of the /dev/shm volume for this step. + This parameter maps to the --shm-size option to docker run. + """ + + name = "kubernetes" + defaults = { + "cpu": "1", + "gpu": "0", + "memory": "4096", + "image": None, + "shared_memory": None, + } + package_url = None + package_sha = None + run_time_limit = None + + def __init__(self, attributes=None, statically_defined=False): + super(KubernetesDecorator, self).__init__( + attributes, statically_defined + ) + + # TODO: Unify the logic + # If no docker image is explicitly specified, impute a default image. + if not self.attributes["image"]: + # If metaflow-config specifies a docker image, just use that. + if BATCH_CONTAINER_IMAGE: + self.attributes["image"] = BATCH_CONTAINER_IMAGE + # If metaflow-config doesn't specify a docker image, assign a + # default docker image. + else: + # Metaflow-R has it's own default docker image (rocker family) + if R.use_r(): + self.attributes["image"] = R.container_image() + # Default to vanilla Python image corresponding to major.minor + # version of the Python interpreter launching the flow. + else: + self.attributes["image"] = "python:%s.%s" % ( + platform.python_version_tuple()[0], + platform.python_version_tuple()[1], + ) + # Assign docker registry URL for the image. + if not get_docker_registry(self.attributes["image"]): + if BATCH_CONTAINER_REGISTRY: + self.attributes["image"] = "%s/%s" % ( + BATCH_CONTAINER_REGISTRY.rstrip("/"), + self.attributes["image"], + ) + + # Refer https://github.com/Netflix/metaflow/blob/master/docs/lifecycle.png + # to understand where these functions are invoked in the lifecycle of a + # Metaflow flow. + def step_init( + self, flow, graph, step, decos, environment, datastore, logger + ): + # Executing Kubernetes jobs requires a non-local datastore at the + # moment. + # TODO: To support MiniKube we need to enable local datastore execution. + if datastore.TYPE != "s3": + raise KubernetesException( + "The *@kubernetes* decorator requires --datastore=s3 " + "at the moment." + ) + + # Set internal state. + self.logger = logger + self.environment = environment + self.step = step + for deco in decos: + if isinstance(deco, ResourcesDecorator): + for k, v in deco.attributes.items(): + # We use the larger of @resources and @k8s attributes + # TODO: Fix https://github.com/Netflix/metaflow/issues/467 + my_val = self.attributes.get(k) + if not (my_val is None and v is None): + self.attributes[k] = str( + max(int(my_val or 0), int(v or 0)) + ) + + # Set run time limit for the Kubernetes job. + self.run_time_limit = get_run_time_limit_for_task(decos) + if self.run_time_limit < 60: + raise KubernetesException( + "The timeout for step *{step}* should be " + "at least 60 seconds for execution on " + "Kubernetes.".format(step=step) + ) + + def runtime_init(self, flow, graph, package, run_id): + # Set some more internal state. + self.flow = flow + self.graph = graph + self.package = package + self.run_id = run_id + + def runtime_task_created( + self, + datastore, + task_id, + split_index, + input_paths, + is_cloned, + ubf_context, + ): + # To execute the Kubernetes job, the job container needs to have + # access to the code package. We store the package in the datastore + # which the pod is able to download as part of it's entrypoint. + if not is_cloned and self.package_url is None: + self.package_url = datastore.save_data( + self.package.sha, TransformableObject(self.package.blob) + ) + self.package_sha = self.package.sha + + def runtime_step_cli( + self, cli_args, retry_count, max_user_code_retries, ubf_context + ): + if retry_count <= max_user_code_retries: + # After all attempts to run the user code have failed, we don't need + # to execute on Kubernetes anymore. We can execute possible fallback + # code locally. + cli_args.commands = ["kubernetes", "step"] + cli_args.command_args.append(self.package_sha) + cli_args.command_args.append(self.package_url) + cli_args.command_options.update(self.attributes) + cli_args.command_options["run-time-limit"] = self.run_time_limit + if not R.use_r(): + cli_args.entrypoint[0] = sys.executable + + def task_pre_step( + self, + step_name, + datastore, + metadata, + run_id, + task_id, + flow, + graph, + retry_count, + max_retries, + ubf_context, + ): + self.metadata = metadata + self.datastore = datastore + + # task_pre_step may run locally if fallback is activated for @catch + # decorator. In that scenario, we skip collecting Kubernetes execution + # metadata. A rudimentary way to detect non-local execution is to + # check for the existence of FOO environment variable. + + if "FOO" in os.environ: + # TODO: Emit k8s related metadata + + # Start MFLog sidecar to collect task logs. + self._save_logs_sidecar = SidecarSubProcess( + "save_logs_periodically" + ) + + def task_finished( + self, step_name, flow, graph, is_task_ok, retry_count, max_retries + ): + # task_finished may run locally if fallback is activated for @catch + # decorator. + if "FOO" in os.environ: + # If `local` metadata is configured, we would need to copy task + # execution metadata from the Kubernetes container to user's + # local file system after the user code has finished execution. + # This happens via datastore as a communication bridge. + if self.metadata.TYPE == "local": + # Note that the datastore is *always* Amazon S3 (see + # runtime_task_created function). + sync_metadata_to_S3( + DATASTORE_LOCAL_DIR, self.datastore.root, retry_count + ) + + try: + self._save_logs_sidecar.kill() + except: + # Best effort kill + pass From ff12dfe3dbb230ee5ef60da066ed90fad3073c63 Mon Sep 17 00:00:00 2001 From: savin Date: Thu, 12 Aug 2021 21:57:16 -0700 Subject: [PATCH 08/32] Kubernetes --- metaflow/plugins/aws/batch/batch_cli.py | 2 +- metaflow/plugins/aws/eks/kubernetes.py | 269 +++++++-------- metaflow/plugins/aws/eks/kubernetes_cli.py | 107 +++--- metaflow/plugins/aws/eks/kubernetes_client.py | 308 +++++++++++++----- .../plugins/aws/eks/kubernetes_decorator.py | 46 ++- 5 files changed, 409 insertions(+), 323 deletions(-) diff --git a/metaflow/plugins/aws/batch/batch_cli.py b/metaflow/plugins/aws/batch/batch_cli.py index e4d99cc5595..ffaf5d9fe76 100644 --- a/metaflow/plugins/aws/batch/batch_cli.py +++ b/metaflow/plugins/aws/batch/batch_cli.py @@ -294,7 +294,7 @@ def _sync_metadata(): attrs=attrs, ) except Exception as e: - print(e) + traceback.print_exc() _sync_metadata() sys.exit(METAFLOW_EXIT_DISALLOW_RETRY) try: diff --git a/metaflow/plugins/aws/eks/kubernetes.py b/metaflow/plugins/aws/eks/kubernetes.py index a65c60a411d..29ba9a7c65e 100644 --- a/metaflow/plugins/aws/eks/kubernetes.py +++ b/metaflow/plugins/aws/eks/kubernetes.py @@ -23,6 +23,7 @@ bash_capture_logs, update_delay, BASH_SAVE_LOGS, + TASK_LOG_SOURCE, ) from metaflow.mflog.mflog import refine, set_should_persist @@ -50,42 +51,47 @@ def __init__( datastore, metadata, environment, + flow_name, + run_id, + step_name, + task_id, + attempt, ): - self.datastore = datastore - self.metadata = metadata - self.environment = environment + self._datastore = datastore + self._metadata = metadata + self._environment = environment + + self._flow_name = flow_name + self._run_id = run_id + self._step_name = step_name + self._task_id = task_id + self._attempt = str(attempt) - # TODO: Issue a kill request for all pending Kubernetes Batch jobs at exit. + # TODO: Issue a kill request for all pending Kubernetes jobs at exit. # atexit.register( # lambda: self.job.kill() if hasattr(self, 'job') else None) def _command( self, - flow_name, - run_id, - step_name, - task_id, - attempt, code_package_url, step_cmds, ): - print("lo") mflog_expr = export_mflog_env_vars( - flow_name=flow_name, - run_id=run_id, - step_name=step_name, - task_id=task_id, - retry_count=attempt, - datastore_type=self.datastore.TYPE, + flow_name=self._flow_name, + run_id=self._run_id, + step_name=self._step_name, + task_id=self._task_id, + retry_count=self._attempt, + datastore_type=self._datastore.TYPE, stdout_path=STDOUT_PATH, stderr_path=STDERR_PATH, ) - return ["echo", "hello"] - init_cmds = self.environment.get_package_commands(code_package_url) + init_cmds = self._environment.get_package_commands(code_package_url) init_expr = " && ".join(init_cmds) step_expr = bash_capture_logs( " && ".join( - self.environment.bootstrap_commands(step_name) + step_cmds + self._environment.bootstrap_commands(self._step_name) + + step_cmds ) ) @@ -109,38 +115,22 @@ def _command( # Note that if step_expr OOMs, this tail expression is never executed. # We lose the last logs in this scenario. # - # TODO: Find a way to capture hard exit logs. + # TODO: Find a way to capture hard exit logs in Kubernetes. cmd_str += "c=$?; %s; exit $c" % BASH_SAVE_LOGS - print("lo") return shlex.split('bash -c "%s"' % cmd_str) - def _name(self, user, flow_name, run_id, step_name, task_id, attempt): - print("hi") - return ( - "{user}-{flow_name}-{run_id}-" - "{step_name}-{task_id}-{attempt}".format( - user=user, - flow_name=flow_name, - run_id=str(run_id) if run_id is not None else "", - step_name=step_name, - task_id=str(task_id) if task_id is not None else "", - attempt=str(attempt) if attempt is not None else "", - ).lower() - ) + def launch_job(self, **kwargs): + self._job = self.create_job(**kwargs).execute() def create_job( self, user, - flow_name, - run_id, - step_name, - task_id, - attempt, code_package_sha, code_package_url, code_package_ds, step_cli, docker_image, + namespace=None, service_account=None, cpu=None, gpu=None, @@ -148,113 +138,92 @@ def create_job( run_time_limit=None, env={}, ): + # TODO: Test for DNS-1123 compliance. + # + # Set the pathspec (along with attempt) as the Kubernetes job name. + # Kubernetes job names are supposed to be unique within a Kubernetes + # namespace and compliant with DNS-1123. The pathspec (with attempt) + # can provide that guarantee, however, for flows launched via AWS Step + # Functions (and potentially Argo), we may not get the task_id or the + # attempt_id while submitting the job to the Kubernetes cluster. If + # that is indeed the case, we can rely on Kubernetes to generate a name + # for us. + job_name = "-".join( + [ + self._flow_name, + self._run_id, + self._step_name, + self._task_id, + self._attempt, + ] + ).lower() + job = ( KubernetesClient() - .job() - .create( - name=self._name( - user=user, - flow_name=flow_name, - run_id=run_id, - step_name=step_name, - task_id=task_id, - attempt=attempt, - ) + .job( + name=job_name, + namespace=namespace, + service_account=service_account, + command=self._command( + code_package_url=code_package_url, + step_cmds=[step_cli], + ), + image=docker_image, + cpu=cpu, + memory=memory, + timeout_in_seconds=run_time_limit, + # Retries are handled by Metaflow runtime + retries=0, ) - ) - - print(job) - print("lk") - # job.name("hello") - print("lk1") - print(job) - print("yahoo") - job.name( - self._name( - user=user, - flow_name=flow_name, - run_id=run_id, - step_name=step_name, - task_id=task_id, - attempt=attempt, + .environment_variable( + # This is needed since `boto3` is not smart enough to figure out + # AWS region by itself. + "AWS_DEFAULT_REGION", + "us-west-2", + ) + .environment_variable("METAFLOW_CODE_SHA", code_package_sha) + .environment_variable("METAFLOW_CODE_URL", code_package_url) + .environment_variable("METAFLOW_CODE_DS", code_package_ds) + .environment_variable("METAFLOW_USER", user) + .environment_variable( + "METAFLOW_SERVICE_URL", BATCH_METADATA_SERVICE_URL + ) + .environment_variable( + "METAFLOW_SERVICE_HEADERS", + json.dumps(BATCH_METADATA_SERVICE_HEADERS), ) - ).command( - self._command( - flow_name=flow_name, - run_id=run_id, - step_name=step_name, - task_id=task_id, - attempt=attempt, - code_package_url=code_package_url, - step_cmds=[step_cli], + .environment_variable( + "METAFLOW_DATASTORE_SYSROOT_S3", DATASTORE_SYSROOT_S3 ) - ).image( - docker_image - ).service_account( - "s3-full-access" - ).cpu( - cpu - ).gpu( - gpu - ).memory( - memory - ).timeout_in_secs( - run_time_limit - ).environment_variable( - "AWS_DEFAULT_REGION", "us-west-2" - ).environment_variable( - "METAFLOW_CODE_SHA", code_package_sha - ).environment_variable( - "METAFLOW_CODE_URL", code_package_url - ).environment_variable( - "METAFLOW_CODE_DS", code_package_ds - ).environment_variable( - "METAFLOW_USER", user - ).environment_variable( - "METAFLOW_SERVICE_URL", BATCH_METADATA_SERVICE_URL - ).environment_variable( - "METAFLOW_SERVICE_HEADERS", - json.dumps(BATCH_METADATA_SERVICE_HEADERS), - ).environment_variable( - "METAFLOW_DATASTORE_SYSROOT_S3", DATASTORE_SYSROOT_S3 - ).environment_variable( - "METAFLOW_DATATOOLS_S3ROOT", DATATOOLS_S3ROOT - ).environment_variable( - "METAFLOW_DEFAULT_DATASTORE", "s3" - ).environment_variable( - "METAFLOW_DEFAULT_METADATA", DEFAULT_METADATA - ).annotation( - "metaflow-flow-name", flow_name - ).annotation( - "metaflow-run-id", run_id - ).annotation( - "metaflow-step-name", step_name - ).annotation( - "metaflow-task-id", task_id - ).annotation( - "metaflow-attempt", attempt - ).annotation( - "metaflow-user", user + .environment_variable("METAFLOW_DATATOOLS_S3ROOT", DATATOOLS_S3ROOT) + .environment_variable("METAFLOW_DEFAULT_DATASTORE", "s3") + .environment_variable("METAFLOW_DEFAULT_METADATA", DEFAULT_METADATA) + .environment_variable("METAFLOW_KUBERNETES_WORKLOAD", 1) ) - # Skip setting METAFLOW_DATASTORE_SYSROOT_LOCAL because metadata - # sync between the local user instance and the remote Kubernetes - # instance assumes metadata is stored in DATASTORE_LOCAL_DIR - # on the remote Kubernetes instance; this happens when - # METAFLOW_DATASTORE_SYSROOT_LOCAL is NOT set (see - # get_datastore_root_from_config in datastore/local.py). + + # Skip setting METAFLOW_DATASTORE_SYSROOT_LOCAL because metadata sync + # between the local user instance and the remote Kubernetes pod + # assumes metadata is stored in DATASTORE_LOCAL_DIR on the Kubernetes + # pod; this happens when METAFLOW_DATASTORE_SYSROOT_LOCAL is NOT set ( + # see get_datastore_root_from_config in datastore/local.py). for name, value in env.items(): job.environment_variable(name, value) - # A Container in a Pod may fail for a number of reasons, such as - # because the process in it exited with a non-zero exit code, or the - # Container was killed due to OOM etc. If this happens, fail the pod - # and let Metaflow handle the retries. - job.restart_policy("Never").retries(0) + # TODO: Add labels and annotations - return job + return job.create() - def wait(self, job, stdout_location, stderr_location, echo=None): - self.job = job + def wait(self, echo=None): + ds = self._datastore( + mode="w", + flow_name=self._flow_name, + run_id=self._run_id, + step_name=self._step_name, + task_id=self._task_id, + attempt=int(self._attempt), + ) + stdout_location = ds.get_log_location(TASK_LOG_SOURCE, "stdout") + stderr_location = ds.get_log_location(TASK_LOG_SOURCE, "stderr") def wait_for_launch(job): status = job.status @@ -273,14 +242,13 @@ def wait_for_launch(job): job_id=job.id, ) t = time.time() - if job.is_running or job.is_done or job.is_crashed: + if job.is_running or job.is_done: break select.poll().poll(200) - prefix = b"[%s] " % util.to_bytes(self.job.id) - def _print_available(tail, stream, should_persist=False): # print the latest batch of lines from S3Tail + prefix = b"[%s] " % util.to_bytes(self._job.id) try: for line in tail: if should_persist: @@ -292,14 +260,14 @@ def _print_available(tail, stream, should_persist=False): echo( "[ temporary error in fetching logs: %s ]" % ex, "stderr", - batch_id=self.job.id, + job_id=self._job.id, ) stdout_tail = S3Tail(stdout_location) stderr_tail = S3Tail(stderr_location) # 1) Loop until the job has started - wait_for_launch(self.job) + wait_for_launch(self._job) # 2) Loop until the job has finished start_time = time.time() @@ -314,7 +282,7 @@ def _print_available(tail, stream, should_persist=False): now = time.time() log_update_delay = update_delay(now - start_time) next_log_update = now + log_update_delay - is_running = self.job.is_running + is_running = self._job.is_running # This sleep should never delay log updates. On the other hand, # we should exit this loop when the task has finished without @@ -338,27 +306,26 @@ def _print_available(tail, stream, should_persist=False): # # TODO: AWS CloudWatch fetch logs - if self.job.is_crashed: + if self._job.has_failed: msg = next( msg for msg in [ - self.job.reason, - self.job.status_reason, - "Task crashed.", + self._job.reason, + "Task crashed", ] if msg is not None ) - raise BatchException( - "%s " + raise KubernetesException( + "%s. " "This could be a transient error. " "Use @retry to retry." % msg ) else: - if self.job.is_running: + if self._job.is_running: # Kill the job if it is still running by throwing an exception. - raise BatchException("Task failed!") + raise KubernetesKilledException("Task failed!") echo( - "Task finished with exit code %s." % self.job.status_code, + "Task finished with exit code %s." % self._job.status_code, "stderr", - batch_id=self.job.id, + job_id=self._job.id, ) diff --git a/metaflow/plugins/aws/eks/kubernetes_cli.py b/metaflow/plugins/aws/eks/kubernetes_cli.py index faffd7e9c6b..020256445f7 100644 --- a/metaflow/plugins/aws/eks/kubernetes_cli.py +++ b/metaflow/plugins/aws/eks/kubernetes_cli.py @@ -90,16 +90,23 @@ def echo(msg, stream="stderr", job_id=None): ctx.obj.echo_always(msg, err=(stream == sys.stderr)) node = ctx.obj.graph[step_name] - if ctx.obj.datastore.datastore_root is None: - ctx.obj.datastore.datastore_root = ( - ctx.obj.datastore.get_datastore_root_from_config(echo) - ) + # TODO: Verify if this check is needed anymore? + # if ctx.obj.datastore.datastore_root is None: + # ctx.obj.datastore.datastore_root = ( + # ctx.obj.datastore.get_datastore_root_from_config(echo) + # ) # Construct entrypoint CLI if executable is None: executable = ctx.obj.environment.executable(step_name) - entrypoint = "%s -u %s" % (executable, os.path.basename(sys.argv[0])) + # Set environment + env = {} + env_deco = [deco for deco in node.decorators if deco.name == "environment"] + if env_deco: + env = env_deco[0].attributes["vars"] + + # Set input paths. input_paths = kwargs.get("input_paths") split_vars = None if input_paths: @@ -110,54 +117,16 @@ def echo(msg, stream="stderr", job_id=None): for i in range(0, len(input_paths), max_size) } kwargs["input_paths"] = "".join("${%s}" % s for s in split_vars.keys()) - - step_args = " ".join(util.dict_to_cli_options(kwargs)) - step_cli = u"{entrypoint} {top_args} step {step} {step_args}".format( - entrypoint=entrypoint, - top_args=" ".join(util.dict_to_cli_options(ctx.parent.parent.params)), - step=step_name, - step_args=" ".join(util.dict_to_cli_options(kwargs)), - ) + env.update(split_vars) # Set retry policy. - retry_count = kwargs.get("retry_count", 0) + retry_count = int(kwargs.get("retry_count", 0)) retry_deco = [deco for deco in node.decorators if deco.name == "retry"] minutes_between_retries = None if retry_deco: minutes_between_retries = int( retry_deco[0].attributes.get("minutes_between_retries", 2) ) - - # Set task attributes - task_spec = { - "flow_name": ctx.obj.flow.name, - "step_name": step_name, - "run_id": kwargs["run_id"], - "task_id": kwargs["task_id"], - "retry_count": str(retry_count), - } - attrs = {"metaflow.%s" % k: v for k, v in task_spec.items()} - attrs["metaflow.user"] = util.get_username() - attrs["metaflow.version"] = ctx.obj.environment.get_environment_info()[ - "metaflow_version" - ] - - # Set environment - env_deco = [deco for deco in node.decorators if deco.name == "environment"] - if env_deco: - env = env_deco[0].attributes["vars"] - else: - env = {} - - datastore_root = os.path.join( - ctx.obj.datastore.make_path( - ctx.obj.flow.name, kwargs["run_id"], step_name, kwargs["task_id"] - ) - ) - # Add the environment variables related to the input-paths argument - if split_vars: - env.update(split_vars) - if retry_count: ctx.obj.echo_always( "Sleeping %d minutes before the next retry" @@ -165,46 +134,56 @@ def echo(msg, stream="stderr", job_id=None): ) time.sleep(minutes_between_retries * 60) - # this information is needed for log tailing - spec = task_spec.copy() - spec["attempt"] = int(spec.pop("retry_count")) - # ds = ctx.obj.datastore(mode='w', **spec) - # stdout_location = ds.get_log_location(TASK_LOG_SOURCE, 'stdout') - # stderr_location = ds.get_log_location(TASK_LOG_SOURCE, 'stderr') + datastore_root = os.path.join( + ctx.obj.datastore.make_path( + ctx.obj.flow.name, kwargs["run_id"], step_name, kwargs["task_id"] + ) + ) - kubernetes = Kubernetes( - ctx.obj.datastore, ctx.obj.metadata, ctx.obj.environment + step_cli = u"{entrypoint} {top_args} step {step} {step_args}".format( + entrypoint="%s -u %s" % (executable, os.path.basename(sys.argv[0])), + top_args=" ".join(util.dict_to_cli_options(ctx.parent.parent.params)), + step=step_name, + step_args=" ".join(util.dict_to_cli_options(kwargs)), ) try: + kubernetes = Kubernetes( + datastore=ctx.obj.datastore, + metadata=ctx.obj.metadata, + environment=ctx.obj.environment, + flow_name=ctx.obj.flow.name, + run_id=kwargs["run_id"], + step_name=step_name, + task_id=kwargs["task_id"], + attempt=retry_count, + ) + # Configure and launch Kubernetes job. with ctx.obj.monitor.measure("metaflow.aws.eks.launch_job"): - job = kubernetes.create_job( + kubernetes.launch_job( user=util.get_username(), - flow_name=ctx.obj.flow.name, - run_id=kwargs["run_id"], - step_name=step_name, - task_id=kwargs["task_id"], - attempt=str(retry_count), code_package_sha=code_package_sha, code_package_url=code_package_url, code_package_ds=ctx.obj.datastore.TYPE, step_cli=step_cli, docker_image=image, - service_account=service_account, + namespace="default", # TODO: Fetch from config + service_account="s3-full-access", # service_account, cpu=cpu, gpu=gpu, memory=memory, run_time_limit=run_time_limit, env=env, - ).execute() + ) except Exception as e: - print(e) + traceback.print_exc() sync_metadata_from_S3(ctx.obj.metadata, datastore_root, retry_count) sys.exit(METAFLOW_EXIT_DISALLOW_RETRY) try: - kubernetes.wait(job=job, echo=echo) + # Wait for the Kubernetes job to finish. + kubernetes.wait(echo=echo) except KubernetesKilledException: - # don't retry killed tasks + # Don't retry killed jobs. traceback.print_exc() sync_metadata_from_S3(ctx.obj.metadata, datastore_root, retry_count) sys.exit(METAFLOW_EXIT_DISALLOW_RETRY) diff --git a/metaflow/plugins/aws/eks/kubernetes_client.py b/metaflow/plugins/aws/eks/kubernetes_client.py index f640a7bd2d9..e95f4ffe9c3 100644 --- a/metaflow/plugins/aws/eks/kubernetes_client.py +++ b/metaflow/plugins/aws/eks/kubernetes_client.py @@ -18,21 +18,20 @@ def __init__(self): # TODO (savin): Look into various ways to configure the Kubernetes # client. - - # TODO (savin): Guard kubernetes import within a try..except. try: + # Kubernetes is a soft dependency. from kubernetes import client, config except (NameError, ImportError): raise MetaflowException( - "Could not import module 'kubernetes'. Install kubernetes Python package first." + "Could not import module 'kubernetes'. Install kubernetes " + "Python package (https://pypi.org/project/kubernetes/) first." ) - from kubernetes import client, config config.load_kube_config() - self._client = client.BatchV1Api() + self._client = client - def job(self): - return KubernetesJob(self._client) + def job(self, **kwargs): + return KubernetesJob(self._client, **kwargs) def attach_job(self, job_id): job = RunningJob(job_id, self._client) @@ -44,133 +43,262 @@ class KubernetesJobException(MetaflowException): class KubernetesJob(object): - def __init__(self, client): - # TODO (savin): Guard kubernetes import within a try..except. - # TODO(?) (savin): Remove dependency on Kubernetes Python SDK. - from kubernetes import client, config - + def __init__(self, client, **kwargs): self._client = client - self._job = None - self._namespace = None - - def create( - self, - name, - namespace, - service_account, - annotations, - labels, - env, - command, - image, - cpu, - memory, - retries, - timeout_in_secs, - ): - self._namespace = namespace - self._job = client.V1Job( + self._kwargs = kwargs + + def create(self): + # Check that job attributes are sensible. + + # CPU value should be greater than 0 + if not ( + isinstance(self._kwargs["cpu"], (int, unicode, basestring, float)) + and float(self._kwargs["cpu"]) > 0 + ): + raise KubernetesJobException( + "Invalid CPU value ({}); it should be greater than 0".format( + self._kwargs["cpu"] + ) + ) + + # Memory value should be greater than 0 + if not ( + isinstance(self._kwargs["memory"], (int, unicode, basestring)) + and int(self._kwargs["memory"]) > 0 + ): + raise KubernetesJobException( + "Invalid memory value ({}); it should be greater than 0".format( + self._kwargs["memory"] + ) + ) + + self._job = self._client.V1Job( api_version="batch/v1", kind="Job", - metadata=client.V1ObjectMeta( - annotations=self._annotations, # annotations are for humans - labels=self._labels, # while labels are for kubernetes - name=self._name, # unique within the namespace - namespace=self._namespace, # empty defaults to `default` + metadata=self._client.V1ObjectMeta( + # Annotations are for humans + annotations=self._kwargs.get("annotations", {}), + # While labels are for kubernetes + labels=self._kwargs.get("labels", {}), + name=self._kwargs["name"], # Unique within the namespace + namespace=self._kwargs["namespace"], # Defaults to `default` ), - spec=client.V1JobSpec( - backoff_limit=self._retries, - ttl_seconds_after_finished=0, # delete the job immediately - template=client.V1PodTemplateSpec( - metadata=client.V1ObjectMeta( - annotations=self._annotations, - labels=self._labels, - name=self._name, - namespace=self._namespace, + spec=self._client.V1JobSpec( + backoff_limit=self._kwargs.get("retries", 0), + ttl_seconds_after_finished=0, # Delete the job immediately + template=self._client.V1PodTemplateSpec( + metadata=self._client.V1ObjectMeta( + annotations=self._kwargs.get("annotations", {}), + labels=self._kwargs.get("labels", {}), + name=self._kwargs["name"], + namespace=self._kwargs["namespace"], ), - spec=client.V1PodSpec( + spec=self._client.V1PodSpec( # Timeout is set on the pod - active_deadline_seconds=self._timeout_in_secs, + active_deadline_seconds=self._kwargs[ + "timeout_in_seconds" + ], # affinity=?, # automount_service_account_token=?, containers=[ - client.V1Container( - command=self._command, + self._client.V1Container( + command=self._kwargs["command"], env=[ - client.V1EnvVar(name=k, value=v) - for k, v in self._env.items() + self._client.V1EnvVar(name=k, value=str(v)) + for k, v in self._kwargs.get( + "environment_variables", {} + ).items() + ] + # And some downward API magic. Add (key, value) + # pairs below to make pod metadata available + # within Kubernetes container. + # TODO: Figure out a way to make container + # metadata visible within the container + + [ + self._client.V1EnvVar( + name=k, + value_from=self._client.V1EnvVarSource( + field_ref=self._client.V1ObjectFieldSelector( + field_path=str(v) + ) + ), + ) + for k, v in { + "METAFLOW_KUBERNETES_POD_NAMESPACE": "metadata.namespace", + "METAFLOW_KUBERNETES_POD_NAME": "metadata.name", + "METAFLOW_KUBERNETES_POD_ID": "metadata.uid", + }.items() ], - image=self._image, - name=self._name, - resources=client.V1ResourceRequirements( + image=self._kwargs["image"], + name=self._kwargs["name"], + resources=self._client.V1ResourceRequirements( requests={ - "cpu": self._cpu, - "memory": self._memory, + "cpu": str(self._kwargs["cpu"]), + "memory": "%sM" + % str(self._kwargs["memory"]), } ), ) ], # image_pull_secrets=?, # preemption_policy=?, + # A Container in a Pod may fail for a number of + # reasons, such as because the process in it exited + # with a non-zero exit code, or the Container was + # killed due to OOM etc. If this happens, fail the pod + # and let Metaflow handle the retries. restart_policy="Never", - service_account_name=self._service_account, + service_account_name=self._kwargs["service_account"], # tolerations=?, # volumes=?, ), ), ), ) + return self def execute(self): - config.load_kube_config() + try: + response = ( + self._client.BatchV1Api() + .create_namespaced_job( + body=self._job, namespace=self._kwargs["namespace"] + ) + .to_dict() + ) + return RunningJob( + client=self._client, + name=response["metadata"]["name"], + namespace=response["metadata"]["namespace"], + ) + except self._client.rest.ApiException as e: + raise KubernetesJobException( + "Unable to launch Kubernetes job.\n %s" % str(e) + ) - response = client.BatchV1Api().create_namespaced_job( - body=self._job, namespace=self._namespace - ) - return RunningJob( - response.to_dict()["spec"]["template"]["metadata"]["name"], - self._client, - ) + def namespace(self, namespace): + self._kwargs["namespace"] = namespace + return self def name(self, name): - self._name = name + self._kwargs["name"] = name return self def command(self, command): - self._command = command + self._kwargs["command"] = command return self def image(self, image): - self._image = image + self._kwargs["image"] = image return self def cpu(self, cpu): - if not ( - isinstance(cpu, (int, unicode, basestring, float)) - and float(cpu) > 0 - ): - raise KubernetesJobException( - "Invalid CPU value ({}); it should be greater than 0".format( - cpu - ) - ) - self._cpu = str(cpu) + self._kwargs["cpu"] = cpu return self def memory(self, mem): - if not (isinstance(mem, (int, unicode, basestring)) and int(mem) > 0): - raise KubernetesJobException( - "Invalid memory value ({}); it should be greater than 0".format( - mem - ) - ) - self._memory = str(mem) + "M" + self._kwargs["memory"] = memory return self def environment_variable(self, name, value): - if name in self.env: - raise KubernetesJobException( - "Duplicate environment variable ({})".format(name) + self._kwargs["environment_variables"] = dict( + self._kwargs.get("environment_variables", {}), **{name: value} + ) + return self + + def label(self, name, value): + self._kwargs["labels"] = dict( + self._kwargs.get("labels", {}), **{name: value} + ) + return self + + def annotation(self, name, value): + self._kwargs["annotations"] = dict( + self._kwargs.get("annotations", {}), **{name: value} + ) + return self + + +class RunningJob(object): + + # TODO: Handle V1JobConditions in V1JobStatus properly + + def __init__(self, client, name, namespace): + self._client = client + self._name = name + self._namespace = namespace + + data = self._update() + self._status = data["status"] + self._id = data["metadata"]["uid"] + + def __repr__(self): + return "{}('{}/{}')".format( + self.__class__.__name__, self._namespace, self._name + ) + + def _update(self): + try: + return ( + self._client.BatchV1Api() + .read_namespaced_job_status( + name=self._name, namespace=self._namespace + ) + .to_dict() ) - self._env[name] = str(value) + except self._client.rest.ApiException as e: + # TODO: Handle failures + raise e + + def update(self): + self._status = self._update()["status"] + print(self._status) return self + + @property + def id(self): + return self._id + + @property + def is_done(self): + if (self._status.get("failed") or 0) + ( + self._status.get("succeeded") or 0 + ) != 1: + # If not done yet, reload the state and check again. + self.update() + return (self._status.get("failed") or 0) + ( + self._status.get("succeeded") or 0 + ) == 1 + else: + return True + + @property + def status(self): + if self.is_running: + return "RUNNING" + if self.has_failed: + return "FAILED" + if self.has_succeeded: + return "SUCCEEDED" + # TODO: Is the state ever UNKNOWN? + return "UNKNOWN" + + @property + def is_running(self): + return not self.is_done and ((self._status.get("active") or 0) > 0) + + @property + def has_failed(self): + return self.is_done and ((self._status.get("failed") or 0) > 0) + + @property + def reason(self): + return "foo" + + @property + def has_succeeded(self): + return self.is_done and ((self._status.get("succeeded") or 0) > 0) + + @property + def status_code(self): + return 1 diff --git a/metaflow/plugins/aws/eks/kubernetes_decorator.py b/metaflow/plugins/aws/eks/kubernetes_decorator.py index 0c3cb2aa91b..1d801256c50 100644 --- a/metaflow/plugins/aws/eks/kubernetes_decorator.py +++ b/metaflow/plugins/aws/eks/kubernetes_decorator.py @@ -3,7 +3,6 @@ import platform import requests -from metaflow import R from metaflow import util from metaflow.decorators import StepDecorator from metaflow.datastore.datastore import TransformableObject @@ -68,6 +67,8 @@ def my_step(self): "memory": "4096", "image": None, "shared_memory": None, + # "namespace", + # "service_account" } package_url = None package_sha = None @@ -78,7 +79,7 @@ def __init__(self, attributes=None, statically_defined=False): attributes, statically_defined ) - # TODO: Unify the logic + # TODO: Unify the logic with AWS Batch # If no docker image is explicitly specified, impute a default image. if not self.attributes["image"]: # If metaflow-config specifies a docker image, just use that. @@ -87,16 +88,12 @@ def __init__(self, attributes=None, statically_defined=False): # If metaflow-config doesn't specify a docker image, assign a # default docker image. else: - # Metaflow-R has it's own default docker image (rocker family) - if R.use_r(): - self.attributes["image"] = R.container_image() # Default to vanilla Python image corresponding to major.minor # version of the Python interpreter launching the flow. - else: - self.attributes["image"] = "python:%s.%s" % ( - platform.python_version_tuple()[0], - platform.python_version_tuple()[1], - ) + self.attributes["image"] = "python:%s.%s" % ( + platform.python_version_tuple()[0], + platform.python_version_tuple()[1], + ) # Assign docker registry URL for the image. if not get_docker_registry(self.attributes["image"]): if BATCH_CONTAINER_REGISTRY: @@ -181,8 +178,7 @@ def runtime_step_cli( cli_args.command_args.append(self.package_url) cli_args.command_options.update(self.attributes) cli_args.command_options["run-time-limit"] = self.run_time_limit - if not R.use_r(): - cli_args.entrypoint[0] = sys.executable + cli_args.entrypoint[0] = sys.executable def task_pre_step( self, @@ -203,10 +199,26 @@ def task_pre_step( # task_pre_step may run locally if fallback is activated for @catch # decorator. In that scenario, we skip collecting Kubernetes execution # metadata. A rudimentary way to detect non-local execution is to - # check for the existence of FOO environment variable. - - if "FOO" in os.environ: - # TODO: Emit k8s related metadata + # check for the existence of METAFLOW_KUBERNETES_WORKLOAD environment + # variable. + + if "METAFLOW_KUBERNETES_WORKLOAD" in os.environ: + meta = {} + meta["kubernetes-pod-id"] = os.environ["METAFLOW_KUBERNETES_POD_ID"] + meta["kubernetes-pod-name"] = os.environ[ + "METAFLOW_KUBERNETES_POD_NAME" + ] + meta["kubernetes-pod-namespace"] = os.environ[ + "METAFLOW_KUBERNETES_POD_NAMESPACE" + ] + # meta['kubernetes-job-attempt'] = ? + + entries = [ + MetaDatum(field=k, value=v, type=k, tags=[]) + for k, v in meta.items() + ] + # Register book-keeping metadata for debugging. + metadata.register_metadata(run_id, step_name, task_id, entries) # Start MFLog sidecar to collect task logs. self._save_logs_sidecar = SidecarSubProcess( @@ -218,7 +230,7 @@ def task_finished( ): # task_finished may run locally if fallback is activated for @catch # decorator. - if "FOO" in os.environ: + if "METAFLOW_KUBERNETES_WORKLOAD" in os.environ: # If `local` metadata is configured, we would need to copy task # execution metadata from the Kubernetes container to user's # local file system after the user code has finished execution. From 2590cea040bc4d2c3a1d7ef6ff8454f1e0a3bdc0 Mon Sep 17 00:00:00 2001 From: savin Date: Wed, 18 Aug 2021 17:02:22 -0700 Subject: [PATCH 09/32] More changes --- metaflow/plugins/aws/eks/kubernetes.py | 4 +- metaflow/plugins/aws/eks/kubernetes_cli.py | 4 +- metaflow/plugins/aws/eks/kubernetes_client.py | 100 +++++++++++------- .../plugins/aws/eks/kubernetes_decorator.py | 1 + 4 files changed, 69 insertions(+), 40 deletions(-) diff --git a/metaflow/plugins/aws/eks/kubernetes.py b/metaflow/plugins/aws/eks/kubernetes.py index 29ba9a7c65e..6acc6c600f9 100644 --- a/metaflow/plugins/aws/eks/kubernetes.py +++ b/metaflow/plugins/aws/eks/kubernetes.py @@ -228,7 +228,7 @@ def wait(self, echo=None): def wait_for_launch(job): status = job.status echo( - "Task is starting (status %s)..." % status, + "Task is starting (status %s)..." % status.upper(), "stderr", job_id=job.id, ) @@ -237,7 +237,7 @@ def wait_for_launch(job): if status != job.status or (time.time() - t) > 30: status = job.status echo( - "Task is starting (status %s)..." % status, + "Task is starting (status %s)..." % status.upper(), "stderr", job_id=job.id, ) diff --git a/metaflow/plugins/aws/eks/kubernetes_cli.py b/metaflow/plugins/aws/eks/kubernetes_cli.py index 020256445f7..8607ff8c09d 100644 --- a/metaflow/plugins/aws/eks/kubernetes_cli.py +++ b/metaflow/plugins/aws/eks/kubernetes_cli.py @@ -176,7 +176,9 @@ def echo(msg, stream="stderr", job_id=None): env=env, ) except Exception as e: - traceback.print_exc() + # TODO: Make sure all errors pretty print nicely. + # traceback.print_exc() + print(str(e)) sync_metadata_from_S3(ctx.obj.metadata, datastore_root, retry_count) sys.exit(METAFLOW_EXIT_DISALLOW_RETRY) try: diff --git a/metaflow/plugins/aws/eks/kubernetes_client.py b/metaflow/plugins/aws/eks/kubernetes_client.py index e95f4ffe9c3..73dc4bf3716 100644 --- a/metaflow/plugins/aws/eks/kubernetes_client.py +++ b/metaflow/plugins/aws/eks/kubernetes_client.py @@ -72,18 +72,39 @@ def create(self): ) ) + # TODO(s) + # 1. Find a way to ensure that a pod is cleanly terminated automatically + # if the container fails to start properly (invalid docker image + # etc.) + + # A discerning eye would notice and question the choice of using the + # V1Job construct over the V1Pod construct given that we don't rely on + # any of the V1Job semantics. The only reasons at the moment are - + # 1. It makes the Kubernetes UIs (Octant, Lens) a bit more easy on + # the eyes, although even that can be questioned. + # 2. AWS Step Functions, at the moment (Aug' 21) only supports + # executing Jobs and not Pods as part of it's publicly declared + # API. When we ship the AWS Step Functions integration with EKS, + # it will hopefully lessen our workload. + # + # The current implementation assumes that there is only one unique Pod + # (unique UID) per Metaflow task attempt. self._job = self._client.V1Job( api_version="batch/v1", kind="Job", metadata=self._client.V1ObjectMeta( # Annotations are for humans annotations=self._kwargs.get("annotations", {}), - # While labels are for kubernetes + # While labels are for Kubernetes labels=self._kwargs.get("labels", {}), name=self._kwargs["name"], # Unique within the namespace namespace=self._kwargs["namespace"], # Defaults to `default` ), spec=self._client.V1JobSpec( + # Retries are handled by Metaflow when it is responsible for + # executing the flow. The responsibility is moved to Kubernetes + # when AWS Step Functions / Argo are responsible for the + # execution. backoff_limit=self._kwargs.get("retries", 0), ttl_seconds_after_finished=0, # Delete the job immediately template=self._client.V1PodTemplateSpec( @@ -94,7 +115,7 @@ def create(self): namespace=self._kwargs["namespace"], ), spec=self._client.V1PodSpec( - # Timeout is set on the pod + # Timeout is set on the pod and not the job (important!) active_deadline_seconds=self._kwargs[ "timeout_in_seconds" ], @@ -112,7 +133,8 @@ def create(self): # And some downward API magic. Add (key, value) # pairs below to make pod metadata available # within Kubernetes container. - # TODO: Figure out a way to make container + # + # TODO: Figure out a way to make job # metadata visible within the container + [ self._client.V1EnvVar( @@ -142,6 +164,7 @@ def create(self): ], # image_pull_secrets=?, # preemption_policy=?, + # # A Container in a Pod may fail for a number of # reasons, such as because the process in it exited # with a non-zero exit code, or the Container was @@ -221,16 +244,18 @@ def annotation(self, name, value): class RunningJob(object): - # TODO: Handle V1JobConditions in V1JobStatus properly + # StateMachine implementation for the lifecycle behavior documented in + # https://kubernetes.io/docs/concepts/workloads/pods/pod-lifecycle/ def __init__(self, client, name, namespace): self._client = client self._name = name self._namespace = namespace - data = self._update() - self._status = data["status"] - self._id = data["metadata"]["uid"] + self._pod = None + + self.update() + self._id = self._pod["metadata"]["labels"]["controller-uid"] def __repr__(self): return "{}('{}/{}')".format( @@ -240,19 +265,20 @@ def __repr__(self): def _update(self): try: return ( - self._client.BatchV1Api() - .read_namespaced_job_status( - name=self._name, namespace=self._namespace + self._client.CoreV1Api() + .list_namespaced_pod( + namespace=self._namespace, + label_selector="job-name={}".format(self._name), ) - .to_dict() + .to_dict()["items"][0] ) except self._client.rest.ApiException as e: # TODO: Handle failures raise e def update(self): - self._status = self._update()["status"] - print(self._status) + self._pod = self._update() + # print(self._pod["status"]) return self @property @@ -261,43 +287,43 @@ def id(self): @property def is_done(self): - if (self._status.get("failed") or 0) + ( - self._status.get("succeeded") or 0 - ) != 1: - # If not done yet, reload the state and check again. + def _done(): + return self._pod["status"]["phase"] in ("Succeeded", "Failed") + + if not _done(): + # if not done, check for newer status self.update() - return (self._status.get("failed") or 0) + ( - self._status.get("succeeded") or 0 - ) == 1 - else: - return True + return _done() @property def status(self): - if self.is_running: - return "RUNNING" - if self.has_failed: - return "FAILED" - if self.has_succeeded: - return "SUCCEEDED" - # TODO: Is the state ever UNKNOWN? - return "UNKNOWN" + if not self.is_done: + # if not done, check for newer status (see the implementation of + # self.is_done) + pass + return self._pod["status"]["phase"] @property - def is_running(self): - return not self.is_done and ((self._status.get("active") or 0) > 0) + def has_succeeded(self): + return self.status == "Succeeded" @property def has_failed(self): - return self.is_done and ((self._status.get("failed") or 0) > 0) + return self.status == "Failed" @property - def reason(self): - return "foo" + def is_running(self): + return self.status == "Running" @property - def has_succeeded(self): - return self.is_done and ((self._status.get("succeeded") or 0) > 0) + def container_status(self): + return ( + self._pod["status"].get("container_statuses", [{}])[0].get("state") + ) + + @property + def reason(self): + return "foo" @property def status_code(self): diff --git a/metaflow/plugins/aws/eks/kubernetes_decorator.py b/metaflow/plugins/aws/eks/kubernetes_decorator.py index 1d801256c50..cf9b8d13b7a 100644 --- a/metaflow/plugins/aws/eks/kubernetes_decorator.py +++ b/metaflow/plugins/aws/eks/kubernetes_decorator.py @@ -204,6 +204,7 @@ def task_pre_step( if "METAFLOW_KUBERNETES_WORKLOAD" in os.environ: meta = {} + # TODO: Get kubernetes job id and job name meta["kubernetes-pod-id"] = os.environ["METAFLOW_KUBERNETES_POD_ID"] meta["kubernetes-pod-name"] = os.environ[ "METAFLOW_KUBERNETES_POD_NAME" From a04608a87b1d01d33aa138bd801c7983c7ce024b Mon Sep 17 00:00:00 2001 From: savin Date: Sat, 21 Aug 2021 09:20:53 -0700 Subject: [PATCH 10/32] More changes --- metaflow/plugins/aws/batch/batch.py | 1 - metaflow/plugins/aws/eks/kubernetes.py | 31 +++++++++++++++++-- metaflow/plugins/aws/eks/kubernetes_client.py | 12 ++++--- .../plugins/aws/eks/kubernetes_decorator.py | 1 + 4 files changed, 37 insertions(+), 8 deletions(-) diff --git a/metaflow/plugins/aws/batch/batch.py b/metaflow/plugins/aws/batch/batch.py index c3bcea1b864..f44c7b91f9c 100644 --- a/metaflow/plugins/aws/batch/batch.py +++ b/metaflow/plugins/aws/batch/batch.py @@ -4,7 +4,6 @@ import select import shlex import time -import warnings from metaflow import util from metaflow.datastore.util.s3tail import S3Tail diff --git a/metaflow/plugins/aws/eks/kubernetes.py b/metaflow/plugins/aws/eks/kubernetes.py index 6acc6c600f9..acc4c815a58 100644 --- a/metaflow/plugins/aws/eks/kubernetes.py +++ b/metaflow/plugins/aws/eks/kubernetes.py @@ -5,7 +5,7 @@ import atexit import shlex import time -import warnings +import re from metaflow import util from metaflow.datastore.util.s3tail import S3Tail @@ -199,6 +199,12 @@ def create_job( .environment_variable("METAFLOW_DEFAULT_DATASTORE", "s3") .environment_variable("METAFLOW_DEFAULT_METADATA", DEFAULT_METADATA) .environment_variable("METAFLOW_KUBERNETES_WORKLOAD", 1) + .label("app", "metaflow") + .label("metaflow/flow_name", self._flow_name) + .label("metaflow/run_id", self._run_id) + .label("metaflow/step_name", self._step_name) + .label("metaflow/task_id", self._task_id) + .label("metaflow/attempt", self._attempt) ) # Skip setting METAFLOW_DATASTORE_SYSROOT_LOCAL because metadata sync @@ -209,7 +215,28 @@ def create_job( for name, value in env.items(): job.environment_variable(name, value) - # TODO: Add labels and annotations + # Add labels to the Kubernetes job + # + # Apply recommended labels https://kubernetes.io/docs/concepts/overview/working-with-objects/common-labels/ + # + # TODO: 1. Verify the behavior of high cardinality labels like instance, + # version etc. in the app.kubernetes.io namespace before + # introducing them here. + job.label("app.kubernetes.io/name", "metaflow-task").label( + "app.kubernetes.io/part-of", "metaflow" + ).label("app.kubernetes.io/created-by", user) + # Add Metaflow system tags as labels as well! + # + # TODO 1. Label values must be an empty string or consist of + # alphanumeric characters, '-', '_' or '.', and must start and + # end with an alphanumeric character. Fix the simple regex + # match below. + for sys_tag in self._metadata.sticky_sys_tags: + job.label( + "metaflow/%s" % sys_tag[: sys_tag.index(":")], + re.sub("[^A-Za-z0-9.-_]", ".", sys_tag[sys_tag.index(":") + 1 :]), + ) + # TODO: Add annotations based on https://kubernetes.io/blog/2021/04/20/annotating-k8s-for-humans/ return job.create() diff --git a/metaflow/plugins/aws/eks/kubernetes_client.py b/metaflow/plugins/aws/eks/kubernetes_client.py index 73dc4bf3716..5f89290b2a0 100644 --- a/metaflow/plugins/aws/eks/kubernetes_client.py +++ b/metaflow/plugins/aws/eks/kubernetes_client.py @@ -1,4 +1,5 @@ from collections import defaultdict +import os try: unicode @@ -8,7 +9,6 @@ from metaflow.exception import MetaflowException -# TODO (savin): Ensure that the client works swimmingly well with the sandbox. class KubernetesClient(object): def __init__(self): # TODO (savin): Look into removing the usage of Kubernetes Python SDK @@ -16,8 +16,6 @@ def __init__(self): # aggressively drops support for older kubernetes clusters, continued # dependency on it may bite our users. - # TODO (savin): Look into various ways to configure the Kubernetes - # client. try: # Kubernetes is a soft dependency. from kubernetes import client, config @@ -26,8 +24,12 @@ def __init__(self): "Could not import module 'kubernetes'. Install kubernetes " "Python package (https://pypi.org/project/kubernetes/) first." ) - - config.load_kube_config() + if os.getenv('KUBERNETES_SERVICE_HOST'): + # We’re inside a pod, auth via ServiceAccount assigned to us + config.load_incluster_config() + else: + # Use kubeconfig, likely $HOME/.kube/config + config.load_kube_config() self._client = client def job(self, **kwargs): diff --git a/metaflow/plugins/aws/eks/kubernetes_decorator.py b/metaflow/plugins/aws/eks/kubernetes_decorator.py index cf9b8d13b7a..f959ec6b917 100644 --- a/metaflow/plugins/aws/eks/kubernetes_decorator.py +++ b/metaflow/plugins/aws/eks/kubernetes_decorator.py @@ -67,6 +67,7 @@ def my_step(self): "memory": "4096", "image": None, "shared_memory": None, + "node_pool": None # "namespace", # "service_account" } From 18ea0b8e65d674fa470ca68d9bf661e51a7102e6 Mon Sep 17 00:00:00 2001 From: savin Date: Tue, 24 Aug 2021 13:19:25 -0700 Subject: [PATCH 11/32] more changes --- metaflow/plugins/aws/eks/kubernetes.py | 10 ++- metaflow/plugins/aws/eks/kubernetes_cli.py | 4 +- metaflow/plugins/aws/eks/kubernetes_client.py | 61 +++++++++++++++---- .../plugins/aws/eks/kubernetes_decorator.py | 7 +-- 4 files changed, 60 insertions(+), 22 deletions(-) diff --git a/metaflow/plugins/aws/eks/kubernetes.py b/metaflow/plugins/aws/eks/kubernetes.py index acc4c815a58..c7cbcb50fd5 100644 --- a/metaflow/plugins/aws/eks/kubernetes.py +++ b/metaflow/plugins/aws/eks/kubernetes.py @@ -138,7 +138,11 @@ def create_job( run_time_limit=None, env={}, ): - # TODO: Test for DNS-1123 compliance. + # TODO: Test for DNS-1123 compliance. Python names can have underscores + # which are not valid Kubernetes names. We can potentially make + # the pathspec DNS-1123 compliant by stripping away underscores + # etc. and relying on Kubernetes to attach a suffix to make the + # name unique within a namespace. # # Set the pathspec (along with attempt) as the Kubernetes job name. # Kubernetes job names are supposed to be unique within a Kubernetes @@ -255,7 +259,7 @@ def wait(self, echo=None): def wait_for_launch(job): status = job.status echo( - "Task is starting (status %s)..." % status.upper(), + "Task is starting (status %s)..." % status, "stderr", job_id=job.id, ) @@ -264,7 +268,7 @@ def wait_for_launch(job): if status != job.status or (time.time() - t) > 30: status = job.status echo( - "Task is starting (status %s)..." % status.upper(), + "Task is starting (status %s)..." % status, "stderr", job_id=job.id, ) diff --git a/metaflow/plugins/aws/eks/kubernetes_cli.py b/metaflow/plugins/aws/eks/kubernetes_cli.py index 8607ff8c09d..1e103c9abaf 100644 --- a/metaflow/plugins/aws/eks/kubernetes_cli.py +++ b/metaflow/plugins/aws/eks/kubernetes_cli.py @@ -177,8 +177,8 @@ def echo(msg, stream="stderr", job_id=None): ) except Exception as e: # TODO: Make sure all errors pretty print nicely. - # traceback.print_exc() - print(str(e)) + traceback.print_exc() + # print(str(e)) sync_metadata_from_S3(ctx.obj.metadata, datastore_root, retry_count) sys.exit(METAFLOW_EXIT_DISALLOW_RETRY) try: diff --git a/metaflow/plugins/aws/eks/kubernetes_client.py b/metaflow/plugins/aws/eks/kubernetes_client.py index 5f89290b2a0..3c5fdb58b6b 100644 --- a/metaflow/plugins/aws/eks/kubernetes_client.py +++ b/metaflow/plugins/aws/eks/kubernetes_client.py @@ -25,7 +25,7 @@ def __init__(self): "Python package (https://pypi.org/project/kubernetes/) first." ) if os.getenv('KUBERNETES_SERVICE_HOST'): - # We’re inside a pod, auth via ServiceAccount assigned to us + # We’re inside a pod, authenticate via ServiceAccount assigned to us config.load_incluster_config() else: # Use kubeconfig, likely $HOME/.kube/config @@ -35,10 +35,6 @@ def __init__(self): def job(self, **kwargs): return KubernetesJob(self._client, **kwargs) - def attach_job(self, job_id): - job = RunningJob(job_id, self._client) - return job.update() - class KubernetesJobException(MetaflowException): headline = "Kubernetes job error" @@ -246,9 +242,42 @@ def annotation(self, name, value): class RunningJob(object): - # StateMachine implementation for the lifecycle behavior documented in + # State Machine implementation for the lifecycle behavior documented in # https://kubernetes.io/docs/concepts/workloads/pods/pod-lifecycle/ + # To ascertain the status of V1Job, we peer into the lifecycle status of + # the pod it is responsible for executing. Unfortunately, the `phase` + # attributes (pending, running, succeeded, failed etc.) only provide + # partial answers and the official API conventions guide suggests that + # it may soon be deprecated (however, not anytime soon - see + # https://github.com/kubernetes/kubernetes/issues/7856). `conditions` otoh + # provide a deeper understanding about the state of the pod; however + # conditions are not state machines and can be oscillating - from the + # offical API conventions guide: + # In general, condition values may change back and forth, but some + # condition transitions may be monotonic, depending on the resource and + # condition type. However, conditions are observations and not, + # themselves, state machines, nor do we define comprehensive state + # machines for objects, nor behaviors associated with state + # transitions. The system is level-based rather than edge-triggered, + # and should assume an Open World. + # In this implementation, we synthesize our notion of "phase" state + # machine from `conditions`, since Kubernetes won't do it for us (for + # many good reasons). + # + # + # + # `conditions` can be of the following types - + # 1. (kubelet) Initialized (always True since we don't rely on init + # containers) + # 2. (kubelet) ContainersReady + # 3. (kubelet) Ready (same as ContainersReady since we don't use + # ReadinessGates - + # https://github.com/kubernetes/kubernetes/blob/master/pkg/kubelet/status/generate.go) + # 4. (kube-scheduler) PodScheduled + # (https://github.com/kubernetes/kubernetes/blob/master/pkg/scheduler/scheduler.go) + # 5. (kube-scheduler) Unschedulable + def __init__(self, client, name, namespace): self._client = client self._name = name @@ -257,6 +286,7 @@ def __init__(self, client, name, namespace): self._pod = None self.update() + # Get the V1Job id (controller for the pod) self._id = self._pod["metadata"]["labels"]["controller-uid"] def __repr__(self): @@ -264,7 +294,7 @@ def __repr__(self): self.__class__.__name__, self._namespace, self._name ) - def _update(self): + def _fetch(self): try: return ( self._client.CoreV1Api() @@ -279,8 +309,8 @@ def _update(self): raise e def update(self): - self._pod = self._update() - # print(self._pod["status"]) + self._pod = self._fetch() + print(self._pod["status"].get("container_statuses", [{}])[0].get("state")) return self @property @@ -303,19 +333,24 @@ def status(self): # if not done, check for newer status (see the implementation of # self.is_done) pass - return self._pod["status"]["phase"] + return (self._pod["status"]["phase"], "foo") @property def has_succeeded(self): - return self.status == "Succeeded" + return self.status[0] == "Succeeded" @property def has_failed(self): - return self.status == "Failed" + return self.status[0] == "Failed" @property def is_running(self): - return self.status == "Running" + return self.status[0] == "Running" + + + @property + def is_pending(self): + return self.status[0] == "Pending" @property def container_status(self): diff --git a/metaflow/plugins/aws/eks/kubernetes_decorator.py b/metaflow/plugins/aws/eks/kubernetes_decorator.py index f959ec6b917..6884a7c26ac 100644 --- a/metaflow/plugins/aws/eks/kubernetes_decorator.py +++ b/metaflow/plugins/aws/eks/kubernetes_decorator.py @@ -63,13 +63,12 @@ def my_step(self): name = "kubernetes" defaults = { "cpu": "1", - "gpu": "0", "memory": "4096", "image": None, "shared_memory": None, - "node_pool": None - # "namespace", - # "service_account" + "node_pool": None, + "namespace": None, + "service_account": None, } package_url = None package_sha = None From 38baa274be7d072ad8d420cf4ead6abe34c30e52 Mon Sep 17 00:00:00 2001 From: savin Date: Wed, 25 Aug 2021 19:05:09 -0700 Subject: [PATCH 12/32] some more changes --- metaflow/plugins/aws/eks/kubernetes.py | 18 +++-- metaflow/plugins/aws/eks/kubernetes_cli.py | 47 +++++++++-- metaflow/plugins/aws/eks/kubernetes_client.py | 77 +++++++++++++------ .../plugins/aws/eks/kubernetes_decorator.py | 8 +- 4 files changed, 109 insertions(+), 41 deletions(-) diff --git a/metaflow/plugins/aws/eks/kubernetes.py b/metaflow/plugins/aws/eks/kubernetes.py index c7cbcb50fd5..5ae19e60dac 100644 --- a/metaflow/plugins/aws/eks/kubernetes.py +++ b/metaflow/plugins/aws/eks/kubernetes.py @@ -130,8 +130,10 @@ def create_job( code_package_ds, step_cli, docker_image, - namespace=None, service_account=None, + secrets=None, + node_selector=None, + namespace=None, cpu=None, gpu=None, memory=None, @@ -168,6 +170,8 @@ def create_job( name=job_name, namespace=namespace, service_account=service_account, + secrets=secrets, + node_selector=node_selector, command=self._command( code_package_url=code_package_url, step_cmds=[step_cli], @@ -224,21 +228,23 @@ def create_job( # Apply recommended labels https://kubernetes.io/docs/concepts/overview/working-with-objects/common-labels/ # # TODO: 1. Verify the behavior of high cardinality labels like instance, - # version etc. in the app.kubernetes.io namespace before + # version etc. in the app.kubernetes.io namespace before # introducing them here. job.label("app.kubernetes.io/name", "metaflow-task").label( "app.kubernetes.io/part-of", "metaflow" ).label("app.kubernetes.io/created-by", user) # Add Metaflow system tags as labels as well! - # - # TODO 1. Label values must be an empty string or consist of - # alphanumeric characters, '-', '_' or '.', and must start and + # + # TODO 1. Label values must be an empty string or consist of + # alphanumeric characters, '-', '_' or '.', and must start and # end with an alphanumeric character. Fix the simple regex # match below. for sys_tag in self._metadata.sticky_sys_tags: job.label( "metaflow/%s" % sys_tag[: sys_tag.index(":")], - re.sub("[^A-Za-z0-9.-_]", ".", sys_tag[sys_tag.index(":") + 1 :]), + re.sub( + "[^A-Za-z0-9.-_]", ".", sys_tag[sys_tag.index(":") + 1 :] + ), ) # TODO: Add annotations based on https://kubernetes.io/blog/2021/04/20/annotating-k8s-for-humans/ diff --git a/metaflow/plugins/aws/eks/kubernetes_cli.py b/metaflow/plugins/aws/eks/kubernetes_cli.py index 1e103c9abaf..889c1c48d9d 100644 --- a/metaflow/plugins/aws/eks/kubernetes_cli.py +++ b/metaflow/plugins/aws/eks/kubernetes_cli.py @@ -35,16 +35,42 @@ def kubernetes(): @click.argument("step-name") @click.argument("code-package-sha") @click.argument("code-package-url") -@click.option("--executable", help="Executable requirement for Kubernetes job.") -@click.option("--image", help="Docker image requirement for Kubernetes job.") +@click.option( + "--executable", + help="Executable requirement for Kubernetes job on Amazon EKS.", +) +@click.option( + "--image", help="Docker image requirement for Kubernetes job on Amazon EKS." +) @click.option( "--service-account", - # TODO: Support more auth mechanisms besides IRSA help="IRSA requirement for Kubernetes job on Amazon EKS.", ) -@click.option("--cpu", help="CPU requirement for Kubernetes job.") -@click.option("--gpu", help="GPU requirement for Kubernetes job.") -@click.option("--memory", help="Memory requirement for Kubernetes job.") +@click.option( + "--secrets", + multiple=True, + default=None, + help="Secrets for Kubernetes job on Amazon EKS.", +) +@click.option( + "--node-selector", + multiple=True, + default=None, + help="NodeSelector for Kubernetes job on Amazon EKS.", +) +@click.option( + # Note that ideally we would have liked to use `namespace` rather than + # `name-space` but unfortunately, `namespace` is already reserved for + # Metaflow namespaces. + "--name-space", + default=None, + help="Namespace for Kubernetes job on Amazon EKS.", +) +@click.option("--cpu", help="CPU requirement for Kubernetes job on Amazon EKS.") +@click.option("--gpu", help="GPU requirement for Kubernetes job on Amazon EKS.") +@click.option( + "--memory", help="Memory requirement for Kubernetes job on Amazon EKS." +) @click.option("--run-id", help="Passed to the top-level 'step'.") @click.option("--task-id", help="Passed to the top-level 'step'.") @click.option("--input-paths", help="Passed to the top-level 'step'.") @@ -77,6 +103,9 @@ def step( executable=None, image=None, service_account=None, + secrets=None, + node_selector=None, + name_space=None, cpu=None, gpu=None, memory=None, @@ -167,8 +196,10 @@ def echo(msg, stream="stderr", job_id=None): code_package_ds=ctx.obj.datastore.TYPE, step_cli=step_cli, docker_image=image, - namespace="default", # TODO: Fetch from config - service_account="s3-full-access", # service_account, + service_account=service_account, + secrets=secrets, + node_selector=node_selector, + namespace=name_space, cpu=cpu, gpu=gpu, memory=memory, diff --git a/metaflow/plugins/aws/eks/kubernetes_client.py b/metaflow/plugins/aws/eks/kubernetes_client.py index 3c5fdb58b6b..06fabef0000 100644 --- a/metaflow/plugins/aws/eks/kubernetes_client.py +++ b/metaflow/plugins/aws/eks/kubernetes_client.py @@ -1,4 +1,3 @@ -from collections import defaultdict import os try: @@ -9,6 +8,7 @@ from metaflow.exception import MetaflowException + class KubernetesClient(object): def __init__(self): # TODO (savin): Look into removing the usage of Kubernetes Python SDK @@ -24,11 +24,16 @@ def __init__(self): "Could not import module 'kubernetes'. Install kubernetes " "Python package (https://pypi.org/project/kubernetes/) first." ) - if os.getenv('KUBERNETES_SERVICE_HOST'): + if os.getenv("KUBERNETES_SERVICE_HOST"): # We’re inside a pod, authenticate via ServiceAccount assigned to us config.load_incluster_config() else: # Use kubeconfig, likely $HOME/.kube/config + # TODO (savin): + # 1. Support generating kubeconfig on the fly using boto3 + # 2. Support auth via OIDC - https://docs.aws.amazon.com/eks/latest/userguide/authenticate-oidc-identity-provider.html + # Supporting the above auth mechanisms (atleast 1.) should be + # good enough for the initial rollout. config.load_kube_config() self._client = client @@ -45,6 +50,9 @@ def __init__(self, client, **kwargs): self._client = client self._kwargs = kwargs + # Kubernetes namespace defaults to `default` + self._kwargs["namespace"] = self._kwargs["namespace"] or "default" + def create(self): # Check that job attributes are sensible. @@ -69,11 +77,11 @@ def create(self): self._kwargs["memory"] ) ) - # TODO(s) # 1. Find a way to ensure that a pod is cleanly terminated automatically # if the container fails to start properly (invalid docker image # etc.) + # 2. Add support for GPUs. # A discerning eye would notice and question the choice of using the # V1Job construct over the V1Pod construct given that we don't rely on @@ -118,7 +126,6 @@ def create(self): "timeout_in_seconds" ], # affinity=?, - # automount_service_account_token=?, containers=[ self._client.V1Container( command=self._kwargs["command"], @@ -149,6 +156,14 @@ def create(self): "METAFLOW_KUBERNETES_POD_ID": "metadata.uid", }.items() ], + env_from=[ + self._client.V1EnvFromSource( + secret_ref=self._client.V1SecretEnvSource( + name=str(k) + ) + ) + for k in self._kwargs.get("secrets", []) + ], image=self._kwargs["image"], name=self._kwargs["name"], resources=self._client.V1ResourceRequirements( @@ -160,7 +175,18 @@ def create(self): ), ) ], + node_selector={ + # TODO: What should be the format of node selector - + # key:value or key=value? + str(k.split("=", 1)[0]): str(k.split("=", 1)[1]) + for k in self._kwargs.get("node_selector", []) + }, + # TODO (savin): At some point in the very near future, + # support docker access secrets. # image_pull_secrets=?, + # + # TODO (savin): We should, someday, get into the pod + # priority business # preemption_policy=?, # # A Container in a Pod may fail for a number of @@ -247,36 +273,38 @@ class RunningJob(object): # To ascertain the status of V1Job, we peer into the lifecycle status of # the pod it is responsible for executing. Unfortunately, the `phase` - # attributes (pending, running, succeeded, failed etc.) only provide + # attributes (pending, running, succeeded, failed etc.) only provide # partial answers and the official API conventions guide suggests that - # it may soon be deprecated (however, not anytime soon - see + # it may soon be deprecated (however, not anytime soon - see # https://github.com/kubernetes/kubernetes/issues/7856). `conditions` otoh - # provide a deeper understanding about the state of the pod; however - # conditions are not state machines and can be oscillating - from the + # provide a deeper understanding about the state of the pod; however + # conditions are not state machines and can be oscillating - from the # offical API conventions guide: # In general, condition values may change back and forth, but some # condition transitions may be monotonic, depending on the resource and - # condition type. However, conditions are observations and not, - # themselves, state machines, nor do we define comprehensive state - # machines for objects, nor behaviors associated with state - # transitions. The system is level-based rather than edge-triggered, + # condition type. However, conditions are observations and not, + # themselves, state machines, nor do we define comprehensive state + # machines for objects, nor behaviors associated with state + # transitions. The system is level-based rather than edge-triggered, # and should assume an Open World. - # In this implementation, we synthesize our notion of "phase" state - # machine from `conditions`, since Kubernetes won't do it for us (for + # In this implementation, we synthesize our notion of "phase" state + # machine from `conditions`, since Kubernetes won't do it for us (for # many good reasons). # # # - # `conditions` can be of the following types - + # `conditions` can be of the following types - # 1. (kubelet) Initialized (always True since we don't rely on init # containers) # 2. (kubelet) ContainersReady - # 3. (kubelet) Ready (same as ContainersReady since we don't use + # 3. (kubelet) Ready (same as ContainersReady since we don't use # ReadinessGates - # https://github.com/kubernetes/kubernetes/blob/master/pkg/kubelet/status/generate.go) - # 4. (kube-scheduler) PodScheduled + # 4. (kube-scheduler) PodScheduled # (https://github.com/kubernetes/kubernetes/blob/master/pkg/scheduler/scheduler.go) # 5. (kube-scheduler) Unschedulable + # + # WIP... def __init__(self, client, name, namespace): self._client = client @@ -310,7 +338,9 @@ def _fetch(self): def update(self): self._pod = self._fetch() - print(self._pod["status"].get("container_statuses", [{}])[0].get("state")) + # print( + # self._pod["status"].get("container_statuses", [{}])[0].get("state") + # ) return self @property @@ -333,24 +363,23 @@ def status(self): # if not done, check for newer status (see the implementation of # self.is_done) pass - return (self._pod["status"]["phase"], "foo") + return self._pod["status"]["phase"] @property def has_succeeded(self): - return self.status[0] == "Succeeded" + return self.status == "Succeeded" @property def has_failed(self): - return self.status[0] == "Failed" + return self.status == "Failed" @property def is_running(self): - return self.status[0] == "Running" - + return self.status == "Running" @property def is_pending(self): - return self.status[0] == "Pending" + return self.status == "Pending" @property def container_status(self): diff --git a/metaflow/plugins/aws/eks/kubernetes_decorator.py b/metaflow/plugins/aws/eks/kubernetes_decorator.py index 6884a7c26ac..c36e1143155 100644 --- a/metaflow/plugins/aws/eks/kubernetes_decorator.py +++ b/metaflow/plugins/aws/eks/kubernetes_decorator.py @@ -65,10 +65,12 @@ def my_step(self): "cpu": "1", "memory": "4096", "image": None, - "shared_memory": None, - "node_pool": None, - "namespace": None, "service_account": None, + "secrets": None, # e.g., mysecret + "node_selector": None, # e.g., kubernetes.io/os=linux + "gpu": "0", + "shared_memory": None, + "name_space": None, } package_url = None package_sha = None From fc0bac472c8f117bbd5b286c5d72f436c018ac36 Mon Sep 17 00:00:00 2001 From: savin Date: Wed, 25 Aug 2021 19:07:56 -0700 Subject: [PATCH 13/32] more changes --- metaflow/plugins/aws/batch/batch.py | 104 +++++++++++++--------------- 1 file changed, 49 insertions(+), 55 deletions(-) diff --git a/metaflow/plugins/aws/batch/batch.py b/metaflow/plugins/aws/batch/batch.py index f44c7b91f9c..2cebdb96c94 100644 --- a/metaflow/plugins/aws/batch/batch.py +++ b/metaflow/plugins/aws/batch/batch.py @@ -187,62 +187,56 @@ def create_job( attrs.get("metaflow.task_id"), attrs.get("metaflow.retry_count"), ) - job = self._client.job() - job.job_name(job_name).job_queue(queue).command( - self._command( - self.environment, - code_package_url, - step_name, - [step_cli], - task_spec, + job = ( + self._client.job() + .job_name(job_name) + .job_queue(queue) + .command( + self._command( + self.environment, + code_package_url, + step_name, + [step_cli], + task_spec, + ) ) - ).image(image).iam_role(iam_role).execution_role( - execution_role - ).job_def( - image, - iam_role, - queue, - execution_role, - shared_memory, - max_swap, - swappiness, - ).cpu( - cpu - ).gpu( - gpu - ).memory( - memory - ).shared_memory( - shared_memory - ).max_swap( - max_swap - ).swappiness( - swappiness - ).timeout_in_secs( - run_time_limit - ).environment_variable( - "AWS_DEFAULT_REGION", self._client.region() - ).environment_variable( - "METAFLOW_CODE_SHA", code_package_sha - ).environment_variable( - "METAFLOW_CODE_URL", code_package_url - ).environment_variable( - "METAFLOW_CODE_DS", code_package_ds - ).environment_variable( - "METAFLOW_USER", attrs["metaflow.user"] - ).environment_variable( - "METAFLOW_SERVICE_URL", BATCH_METADATA_SERVICE_URL - ).environment_variable( - "METAFLOW_SERVICE_HEADERS", - json.dumps(BATCH_METADATA_SERVICE_HEADERS), - ).environment_variable( - "METAFLOW_DATASTORE_SYSROOT_S3", DATASTORE_SYSROOT_S3 - ).environment_variable( - "METAFLOW_DATATOOLS_S3ROOT", DATATOOLS_S3ROOT - ).environment_variable( - "METAFLOW_DEFAULT_DATASTORE", "s3" - ).environment_variable( - "METAFLOW_DEFAULT_METADATA", DEFAULT_METADATA + .image(image) + .iam_role(iam_role) + .execution_role(execution_role) + .job_def( + image, + iam_role, + queue, + execution_role, + shared_memory, + max_swap, + swappiness, + ) + .cpu(cpu) + .gpu(gpu) + .memory(memory) + .shared_memory(shared_memory) + .max_swap(max_swap) + .swappiness(swappiness) + .timeout_in_secs(run_time_limit) + .environment_variable("AWS_DEFAULT_REGION", self._client.region()) + .environment_variable("METAFLOW_CODE_SHA", code_package_sha) + .environment_variable("METAFLOW_CODE_URL", code_package_url) + .environment_variable("METAFLOW_CODE_DS", code_package_ds) + .environment_variable("METAFLOW_USER", attrs["metaflow.user"]) + .environment_variable( + "METAFLOW_SERVICE_URL", BATCH_METADATA_SERVICE_URL + ) + .environment_variable( + "METAFLOW_SERVICE_HEADERS", + json.dumps(BATCH_METADATA_SERVICE_HEADERS), + ) + .environment_variable( + "METAFLOW_DATASTORE_SYSROOT_S3", DATASTORE_SYSROOT_S3 + ) + .environment_variable("METAFLOW_DATATOOLS_S3ROOT", DATATOOLS_S3ROOT) + .environment_variable("METAFLOW_DEFAULT_DATASTORE", "s3") + .environment_variable("METAFLOW_DEFAULT_METADATA", DEFAULT_METADATA) ) # Skip setting METAFLOW_DATASTORE_SYSROOT_LOCAL because metadata sync between the local user # instance and the remote AWS Batch instance assumes metadata is stored in DATASTORE_LOCAL_DIR From e54f79f0a7b8f6c048aaf9128e999ee934f01993 Mon Sep 17 00:00:00 2001 From: savin Date: Wed, 25 Aug 2021 19:35:43 -0700 Subject: [PATCH 14/32] add disk space --- metaflow/plugins/aws/eks/kubernetes.py | 2 ++ metaflow/plugins/aws/eks/kubernetes_cli.py | 3 ++ metaflow/plugins/aws/eks/kubernetes_client.py | 31 +++++++++++++++++-- .../plugins/aws/eks/kubernetes_decorator.py | 2 ++ 4 files changed, 36 insertions(+), 2 deletions(-) diff --git a/metaflow/plugins/aws/eks/kubernetes.py b/metaflow/plugins/aws/eks/kubernetes.py index 5ae19e60dac..a4a4e033f2d 100644 --- a/metaflow/plugins/aws/eks/kubernetes.py +++ b/metaflow/plugins/aws/eks/kubernetes.py @@ -136,6 +136,7 @@ def create_job( namespace=None, cpu=None, gpu=None, + disk=None, memory=None, run_time_limit=None, env={}, @@ -179,6 +180,7 @@ def create_job( image=docker_image, cpu=cpu, memory=memory, + disk=disk, timeout_in_seconds=run_time_limit, # Retries are handled by Metaflow runtime retries=0, diff --git a/metaflow/plugins/aws/eks/kubernetes_cli.py b/metaflow/plugins/aws/eks/kubernetes_cli.py index 889c1c48d9d..61a09685831 100644 --- a/metaflow/plugins/aws/eks/kubernetes_cli.py +++ b/metaflow/plugins/aws/eks/kubernetes_cli.py @@ -68,6 +68,7 @@ def kubernetes(): ) @click.option("--cpu", help="CPU requirement for Kubernetes job on Amazon EKS.") @click.option("--gpu", help="GPU requirement for Kubernetes job on Amazon EKS.") +@click.option("--disk", help="Disk requirement for Kubernetes job on Amazon EKS.") @click.option( "--memory", help="Memory requirement for Kubernetes job on Amazon EKS." ) @@ -108,6 +109,7 @@ def step( name_space=None, cpu=None, gpu=None, + disk=None, memory=None, run_time_limit=None, **kwargs @@ -202,6 +204,7 @@ def echo(msg, stream="stderr", job_id=None): namespace=name_space, cpu=cpu, gpu=gpu, + disk=disk, memory=memory, run_time_limit=run_time_limit, env=env, diff --git a/metaflow/plugins/aws/eks/kubernetes_client.py b/metaflow/plugins/aws/eks/kubernetes_client.py index 06fabef0000..6e2e2fe833a 100644 --- a/metaflow/plugins/aws/eks/kubernetes_client.py +++ b/metaflow/plugins/aws/eks/kubernetes_client.py @@ -11,7 +11,7 @@ class KubernetesClient(object): def __init__(self): - # TODO (savin): Look into removing the usage of Kubernetes Python SDK + # TODO: Look into removing the usage of Kubernetes Python SDK # at some point in the future. Given that Kubernetes Python SDK # aggressively drops support for older kubernetes clusters, continued # dependency on it may bite our users. @@ -77,7 +77,19 @@ def create(self): self._kwargs["memory"] ) ) - # TODO(s) + + # Disk value should be greater than 0 + if not ( + isinstance(self._kwargs["disk"], (int, unicode, basestring)) + and int(self._kwargs["disk"]) > 0 + ): + raise KubernetesJobException( + "Invalid disk value ({}); it should be greater than 0".format( + self._kwargs["disk"] + ) + ) + + # TODO(s) (savin) # 1. Find a way to ensure that a pod is cleanly terminated automatically # if the container fails to start properly (invalid docker image # etc.) @@ -125,6 +137,12 @@ def create(self): active_deadline_seconds=self._kwargs[ "timeout_in_seconds" ], + # TODO (savin): Enable affinities for GPU scheduling. + # This requires some thought around the + # UX since specifying affinities can get + # complicated quickly. We may well decide + # to move it out of scope for the initial + # roll out. # affinity=?, containers=[ self._client.V1Container( @@ -171,6 +189,8 @@ def create(self): "cpu": str(self._kwargs["cpu"]), "memory": "%sM" % str(self._kwargs["memory"]), + "ephemeral-storage": "%sM" + % str(self._kwargs["disk"]), } ), ) @@ -196,7 +216,14 @@ def create(self): # and let Metaflow handle the retries. restart_policy="Never", service_account_name=self._kwargs["service_account"], + # TODO (savin): Enable tolerations for GPU scheduling. + # This requires some thought around the + # UX since specifying tolerations can get + # complicated quickly. # tolerations=?, + # + # TODO (savin): At some point in the very near future, + # support custom volumes (PVCs/EVCs). # volumes=?, ), ), diff --git a/metaflow/plugins/aws/eks/kubernetes_decorator.py b/metaflow/plugins/aws/eks/kubernetes_decorator.py index c36e1143155..35e878473e9 100644 --- a/metaflow/plugins/aws/eks/kubernetes_decorator.py +++ b/metaflow/plugins/aws/eks/kubernetes_decorator.py @@ -25,6 +25,7 @@ class KubernetesDecorator(StepDecorator): """ + TODO (savin): Update this docstring. Step decorator to specify that this step should execute on Kubernetes. This decorator indicates that your step should execute on Kubernetes. Note @@ -64,6 +65,7 @@ def my_step(self): defaults = { "cpu": "1", "memory": "4096", + "disk": "10240", "image": None, "service_account": None, "secrets": None, # e.g., mysecret From 433c1eb59dbf1e93d1b0a75188852ff9198a90e4 Mon Sep 17 00:00:00 2001 From: savin Date: Wed, 25 Aug 2021 20:23:41 -0700 Subject: [PATCH 15/32] Add todos --- metaflow/plugins/aws/eks/kubernetes_client.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/metaflow/plugins/aws/eks/kubernetes_client.py b/metaflow/plugins/aws/eks/kubernetes_client.py index 6e2e2fe833a..67bee9e8428 100644 --- a/metaflow/plugins/aws/eks/kubernetes_client.py +++ b/metaflow/plugins/aws/eks/kubernetes_client.py @@ -225,6 +225,8 @@ def create(self): # TODO (savin): At some point in the very near future, # support custom volumes (PVCs/EVCs). # volumes=?, + # + # TODO (savin): Set termination_message_policy ), ), ), From 1984872ea60eced5aa0a6fe9771dab3e5750d8ba Mon Sep 17 00:00:00 2001 From: savin Date: Thu, 26 Aug 2021 17:24:01 -0700 Subject: [PATCH 16/32] some fixes --- metaflow/plugins/aws/eks/kubernetes.py | 15 +++------------ metaflow/plugins/aws/eks/kubernetes_cli.py | 4 +++- metaflow/plugins/aws/eks/kubernetes_client.py | 19 +++++++++++++++++-- 3 files changed, 23 insertions(+), 15 deletions(-) diff --git a/metaflow/plugins/aws/eks/kubernetes.py b/metaflow/plugins/aws/eks/kubernetes.py index a4a4e033f2d..1a2028b8ba4 100644 --- a/metaflow/plugins/aws/eks/kubernetes.py +++ b/metaflow/plugins/aws/eks/kubernetes.py @@ -2,7 +2,6 @@ import time import json import select -import atexit import shlex import time import re @@ -46,6 +45,7 @@ class KubernetesKilledException(MetaflowException): class Kubernetes(object): + def __init__( self, datastore, @@ -67,10 +67,6 @@ def __init__( self._task_id = task_id self._attempt = str(attempt) - # TODO: Issue a kill request for all pending Kubernetes jobs at exit. - # atexit.register( - # lambda: self.job.kill() if hasattr(self, 'job') else None) - def _command( self, code_package_url, @@ -334,16 +330,11 @@ def _print_available(tail, stream, should_persist=False): # It is possible that we exit the loop above before all logs have been # shown. # - # TODO if we notice AWS Batch failing to upload logs to S3, we can add a - # HEAD request here to ensure that the file exists prior to calling + # TODO if we notice Kubernetes failing to upload logs to S3, we can add + # a HEAD request here to ensure that the file exists prior to calling # S3Tail and note the user about truncated logs if it doesn't _print_available(stdout_tail, "stdout") _print_available(stderr_tail, "stderr") - # In case of hard crashes (OOM), the final save_logs won't happen. - # We fetch the remaining logs from AWS CloudWatch and persist them to - # Amazon S3. - # - # TODO: AWS CloudWatch fetch logs if self._job.has_failed: msg = next( diff --git a/metaflow/plugins/aws/eks/kubernetes_cli.py b/metaflow/plugins/aws/eks/kubernetes_cli.py index 61a09685831..2cdb2d6df44 100644 --- a/metaflow/plugins/aws/eks/kubernetes_cli.py +++ b/metaflow/plugins/aws/eks/kubernetes_cli.py @@ -68,7 +68,9 @@ def kubernetes(): ) @click.option("--cpu", help="CPU requirement for Kubernetes job on Amazon EKS.") @click.option("--gpu", help="GPU requirement for Kubernetes job on Amazon EKS.") -@click.option("--disk", help="Disk requirement for Kubernetes job on Amazon EKS.") +@click.option( + "--disk", help="Disk requirement for Kubernetes job on Amazon EKS." +) @click.option( "--memory", help="Memory requirement for Kubernetes job on Amazon EKS." ) diff --git a/metaflow/plugins/aws/eks/kubernetes_client.py b/metaflow/plugins/aws/eks/kubernetes_client.py index 67bee9e8428..2a01db1370d 100644 --- a/metaflow/plugins/aws/eks/kubernetes_client.py +++ b/metaflow/plugins/aws/eks/kubernetes_client.py @@ -124,7 +124,7 @@ def create(self): # when AWS Step Functions / Argo are responsible for the # execution. backoff_limit=self._kwargs.get("retries", 0), - ttl_seconds_after_finished=0, # Delete the job immediately + ttl_seconds_after_finished=60*60*24, # Remove job after a day. template=self._client.V1PodTemplateSpec( metadata=self._client.V1ObjectMeta( annotations=self._kwargs.get("annotations", {}), @@ -273,7 +273,7 @@ def cpu(self, cpu): return self def memory(self, mem): - self._kwargs["memory"] = memory + self._kwargs["memory"] = mem return self def environment_variable(self, name, value): @@ -344,8 +344,12 @@ def __init__(self, client, name, namespace): self.update() # Get the V1Job id (controller for the pod) + # TODO (savin): Should the id be job id, job name, pod id or pod name? self._id = self._pod["metadata"]["labels"]["controller-uid"] + import atexit + atexit.register(self.kill) + def __repr__(self): return "{}('{}/{}')".format( self.__class__.__name__, self._namespace, self._name @@ -353,6 +357,7 @@ def __repr__(self): def _fetch(self): try: + # TODO (savin): pods may not appear immediately. return ( self._client.CoreV1Api() .list_namespaced_pod( @@ -372,6 +377,16 @@ def update(self): # ) return self + def kill(self): + if not self.is_done: + # TODO (savin): Currently, we are deleting the job. Ideally, we + # should terminate the job without deleting the + # object. + self._client.BatchV1Api().delete_namespaced_job( + name=self._name, namespace=self._namespace, propagation_policy="Background" + ) + return self.update() + @property def id(self): return self._id From 038d9d5c32f94a696edaba178bc76c9ff9da434f Mon Sep 17 00:00:00 2001 From: savin Date: Thu, 26 Aug 2021 17:24:53 -0700 Subject: [PATCH 17/32] add k8s testing context --- test/core/contexts.json | 34 ++++++++++++++++++++++++++++++++++ 1 file changed, 34 insertions(+) diff --git a/test/core/contexts.json b/test/core/contexts.json index cedeac3c36a..562303317d8 100644 --- a/test/core/contexts.json +++ b/test/core/contexts.json @@ -91,6 +91,40 @@ "disabled_tests": [ "S3FailureTest" ] + }, + { + "name": "python3-k8s", + "disabled": true, + "python": "python3", + "top_options": [ + "--event-logger=nullSidecarLogger", + "--no-pylint", + "--quiet", + "--with=kubernetes", + "--datastore=s3" + ], + "env": { + "METAFLOW_USER": "tester", + "METAFLOW_RUN_BOOL_PARAM": "False", + "METAFLOW_RUN_NO_DEFAULT_PARAM": "test_str", + "METAFLOW_DEFAULT_METADATA": "service" + }, + "run_options": [ + "--max-workers", "50", + "--max-num-splits", "10000", + "--tag", "\u523a\u8eab means sashimi", + "--tag", "multiple tags should be ok" + ], + "checks": ["python3-cli", "python3-metadata"], + "disabled_tests": [ + "LargeArtifactTest", + "WideForeachTest", + "TagCatchTest", + "BasicUnboundedForeachTest", + "NestedUnboundedForeachTest", + "DetectSegFaultTest", + "TimeoutDecoratorTest" + ] } ], "checks": { From 6d0fb91930b87ad8e9de82842e3ba4c3435af159 Mon Sep 17 00:00:00 2001 From: savin Date: Fri, 27 Aug 2021 14:39:13 -0700 Subject: [PATCH 18/32] more changes --- metaflow/plugins/aws/batch/batch_client.py | 58 +-------- metaflow/plugins/aws/eks/kubernetes.py | 9 +- metaflow/plugins/aws/eks/kubernetes_client.py | 122 +++++++++++++++--- 3 files changed, 112 insertions(+), 77 deletions(-) diff --git a/metaflow/plugins/aws/batch/batch_client.py b/metaflow/plugins/aws/batch/batch_client.py index fee8ca2e9fe..f2a15b2d226 100644 --- a/metaflow/plugins/aws/batch/batch_client.py +++ b/metaflow/plugins/aws/batch/batch_client.py @@ -463,64 +463,8 @@ def status_code(self): self.update() return self.info['container'].get('exitCode') - def wait_for_running(self): - if not self.is_running and not self.is_done: - BatchWaiter(self._client).wait_for_running(self.id) - def kill(self): if not self.is_done: self._client.terminate_job( jobId=self._id, reason='Metaflow initiated job termination.') - return self.update() - - -class BatchWaiter(object): - def __init__(self, client): - try: - from botocore import waiter - except: - raise BatchJobException( - 'Could not import module \'botocore\' which ' - 'is required for Batch jobs. Install botocore ' - 'first.' - ) - self._client = client - self._waiter = waiter - - def wait_for_running(self, job_id): - model = self._waiter.WaiterModel( - { - 'version': 2, - 'waiters': { - 'JobRunning': { - 'delay': 1, - 'operation': 'DescribeJobs', - 'description': 'Wait until job starts running', - 'maxAttempts': 1000000, - 'acceptors': [ - { - 'argument': 'jobs[].status', - 'expected': 'SUCCEEDED', - 'matcher': 'pathAll', - 'state': 'success', - }, - { - 'argument': 'jobs[].status', - 'expected': 'FAILED', - 'matcher': 'pathAny', - 'state': 'success', - }, - { - 'argument': 'jobs[].status', - 'expected': 'RUNNING', - 'matcher': 'pathAny', - 'state': 'success', - }, - ], - } - }, - } - ) - self._waiter.create_waiter_with_client('JobRunning', model, self._client).wait( - jobs=[job_id] - ) \ No newline at end of file + return self.update() \ No newline at end of file diff --git a/metaflow/plugins/aws/eks/kubernetes.py b/metaflow/plugins/aws/eks/kubernetes.py index 1a2028b8ba4..ecbe77a4343 100644 --- a/metaflow/plugins/aws/eks/kubernetes.py +++ b/metaflow/plugins/aws/eks/kubernetes.py @@ -45,7 +45,6 @@ class KubernetesKilledException(MetaflowException): class Kubernetes(object): - def __init__( self, datastore, @@ -330,9 +329,11 @@ def _print_available(tail, stream, should_persist=False): # It is possible that we exit the loop above before all logs have been # shown. # - # TODO if we notice Kubernetes failing to upload logs to S3, we can add - # a HEAD request here to ensure that the file exists prior to calling - # S3Tail and note the user about truncated logs if it doesn't + # TODO (savin): If we notice Kubernetes failing to upload logs to S3, + # we can add a HEAD request here to ensure that the file + # exists prior to calling S3Tail and note the user about + # truncated logs if it doesn't. + # TODO (savin): For hard crashes, we can fetch logs from the pod. _print_available(stdout_tail, "stdout") _print_available(stderr_tail, "stderr") diff --git a/metaflow/plugins/aws/eks/kubernetes_client.py b/metaflow/plugins/aws/eks/kubernetes_client.py index 2a01db1370d..2b0fc1978b5 100644 --- a/metaflow/plugins/aws/eks/kubernetes_client.py +++ b/metaflow/plugins/aws/eks/kubernetes_client.py @@ -9,6 +9,10 @@ from metaflow.exception import MetaflowException +class KubernetesJobException(MetaflowException): + headline = "Kubernetes job error" + + class KubernetesClient(object): def __init__(self): # TODO: Look into removing the usage of Kubernetes Python SDK @@ -41,10 +45,6 @@ def job(self, **kwargs): return KubernetesJob(self._client, **kwargs) -class KubernetesJobException(MetaflowException): - headline = "Kubernetes job error" - - class KubernetesJob(object): def __init__(self, client, **kwargs): self._client = client @@ -124,7 +124,13 @@ def create(self): # when AWS Step Functions / Argo are responsible for the # execution. backoff_limit=self._kwargs.get("retries", 0), - ttl_seconds_after_finished=60*60*24, # Remove job after a day. + completions=1, # A single non-indexed pod job + # TODO (savin): Implement a job clean-up option in the + # kubernetes CLI. + ttl_seconds_after_finished=7 + * 60 + * 60 # Remove job after a week. TODO (savin): Make this + * 24, # configurable template=self._client.V1PodTemplateSpec( metadata=self._client.V1ObjectMeta( annotations=self._kwargs.get("annotations", {}), @@ -216,6 +222,8 @@ def create(self): # and let Metaflow handle the retries. restart_policy="Never", service_account_name=self._kwargs["service_account"], + # Terminate the container immediately on SIGTERM + termination_grace_period_seconds=0, # TODO (savin): Enable tolerations for GPU scheduling. # This requires some thought around the # UX since specifying tolerations can get @@ -235,6 +243,10 @@ def create(self): def execute(self): try: + # TODO (savin): Make job submission back-pressure aware. Currently + # there doesn't seem to be a kubernetes-native way to + # achieve the guarantees that we are seeking. + # Hopefully, we will be able to get creative soon. response = ( self._client.BatchV1Api() .create_namespaced_job( @@ -348,6 +360,7 @@ def __init__(self, client, name, namespace): self._id = self._pod["metadata"]["labels"]["controller-uid"] import atexit + atexit.register(self.kill) def __repr__(self): @@ -355,37 +368,114 @@ def __repr__(self): self.__class__.__name__, self._namespace, self._name ) - def _fetch(self): + def _fetch_pod(self): try: - # TODO (savin): pods may not appear immediately. + # TODO (savin): pods may not appear immediately or they may + # disappear return ( self._client.CoreV1Api() .list_namespaced_pod( namespace=self._namespace, label_selector="job-name={}".format(self._name), ) - .to_dict()["items"][0] + .to_dict()["items"] + or [None] + )[0] + except self._client.rest.ApiException as e: + # TODO: Handle failures + raise e + + def _fetch_job(self): + try: + return self._client.BatchV1Api().read_namespaced_job( + name=self._name, namespace=self._namespace ) except self._client.rest.ApiException as e: # TODO: Handle failures raise e def update(self): - self._pod = self._fetch() + self._pod = self._fetch_pod() + # print(self._pod) # print( # self._pod["status"].get("container_statuses", [{}])[0].get("state") # ) return self def kill(self): + # Terminating a Kubernetes job is a bit tricky. Issuing a + # `BatchV1Api.delete_namespaced_job` will also remove all traces of the # job object from the Kubernetes API server which may not be desirable. + # This forces us to be a bit creative in terms of how we handle kill: + # + # 1. If the container is alive and kicking inside the pod, we simply + # attach ourselves to the container and issue a kill signal. The + # way we have initialized the Job ensures that the job will cleanly + # terminate. + # 2. In scenarios where either the pod (unschedulable etc.) or the + # container (ImagePullError etc.) hasn't come up yet, we become a + # bit creative by patching the job parallelism to 0. This ensures + # that the underlying node's resources are made available to + # kube-scheduler again. The downside is that the Job wouldn't mark + # itself as done and the pod metadata disappears from the API + # server. There is an open issue in the Kubernetes GH to provide + # better support for job terminations - + # https://github.com/kubernetes/enhancements/issues/2232 but + # meanwhile as a quick follow-up, we should investigate ways to + # terminate the pod without deleting the object. + # 3. If the pod object hasn't shown up yet, we set the parallelism to 0 + # to preempt it. if not self.is_done: - # TODO (savin): Currently, we are deleting the job. Ideally, we - # should terminate the job without deleting the - # object. - self._client.BatchV1Api().delete_namespaced_job( - name=self._name, namespace=self._namespace, propagation_policy="Background" - ) - return self.update() + # TODO (savin): Swap the check with if container is running. + if self.is_running: + # Case 1. + from kubernetes.stream import stream + + api_instance = self._client.CoreV1Api + try: + # TODO (savin): stream opens a web-socket connection. It may + # not be desirable to open multiple web-socket + # connections frivolously (think killing a + # workflow during a for-each step). Given that + # we are only interested in a fire-and-forget + # request, we should look into how to avoid + # the ws. + stream( + api_instance().connect_get_namespaced_pod_exec, + name=self._pod["metadata"]["name"], + namespace=self._namespace, + command=[ + "/bin/sh", + "-c", + "/sbin/killall5", + ], + stderr=True, + stdin=False, + stdout=True, + tty=False, + ) + except: + # Best effort. It's likely that this API call could also be + # blocked for the user. + # TODO (savin): Forward the error to the user. + # pass + raise + else: + # Case 2. + try: + # TODO (savin): Also patch job annotation to reflect this + # action. + self._client.BatchV1Api().patch_namespaced_job( + name=self._name, + namespace=self._namespace, + field_manager="metaflow", + body={"spec": {"parallelism": 0}}, + ) + except: + # Best effort. + # TODO (savin): Forward the error to the user. + # pass + raise + return self @property def id(self): From e615fcc249fdde689d28873c11e65d0805f67cc0 Mon Sep 17 00:00:00 2001 From: savin Date: Fri, 27 Aug 2021 22:53:25 -0700 Subject: [PATCH 19/32] some more changes --- metaflow/plugins/aws/eks/kubernetes.py | 45 +++-- metaflow/plugins/aws/eks/kubernetes_client.py | 187 ++++++++++++------ .../plugins/aws/eks/kubernetes_decorator.py | 2 +- 3 files changed, 154 insertions(+), 80 deletions(-) diff --git a/metaflow/plugins/aws/eks/kubernetes.py b/metaflow/plugins/aws/eks/kubernetes.py index ecbe77a4343..f1acfb30fd4 100644 --- a/metaflow/plugins/aws/eks/kubernetes.py +++ b/metaflow/plugins/aws/eks/kubernetes.py @@ -262,23 +262,24 @@ def wait(self, echo=None): def wait_for_launch(job): status = job.status echo( - "Task is starting (status %s)..." % status, + "Task is starting (Status %s)..." % status, "stderr", job_id=job.id, ) t = time.time() while True: - if status != job.status or (time.time() - t) > 30: - status = job.status + new_status = job.status + if status != new_status or (time.time() - t) > 30: + status = new_status echo( - "Task is starting (status %s)..." % status, + "Task is starting (Status %s)..." % status, "stderr", job_id=job.id, ) t = time.time() if job.is_running or job.is_done: break - select.poll().poll(200) + time.sleep(1) def _print_available(tail, stream, should_persist=False): # print the latest batch of lines from S3Tail @@ -321,8 +322,7 @@ def _print_available(tail, stream, should_persist=False): # This sleep should never delay log updates. On the other hand, # we should exit this loop when the task has finished without # a long delay, regardless of the log tailing schedule - d = min(log_update_delay, 5.0) - select.poll().poll(d * 1000) + time.sleep(min(log_update_delay, 5.0)) # 3) Fetch remaining logs # @@ -338,25 +338,32 @@ def _print_available(tail, stream, should_persist=False): _print_available(stderr_tail, "stderr") if self._job.has_failed: + exit_code, reason = self._job.reason msg = next( msg for msg in [ - self._job.reason, + reason, "Task crashed", ] if msg is not None ) + if exit_code: + if int(exit_code) == 139: + raise KubernetesException( + "Task failed with a segmentation fault." + ) + else: + msg = "%s (exit code %s)" % (msg, exit_code) raise KubernetesException( - "%s. " - "This could be a transient error. " + "%s. This could be a transient error. " "Use @retry to retry." % msg ) - else: - if self._job.is_running: - # Kill the job if it is still running by throwing an exception. - raise KubernetesKilledException("Task failed!") - echo( - "Task finished with exit code %s." % self._job.status_code, - "stderr", - job_id=self._job.id, - ) + elif not self._job.is_done: + # Kill the job if it is still running by throwing an exception. + raise KubernetesKilledException("Task failed!") + exit_code, _ = self._job.reason + echo( + "Task finished with exit code %s." % exit_code, + "stderr", + job_id=self._job.id, + ) diff --git a/metaflow/plugins/aws/eks/kubernetes_client.py b/metaflow/plugins/aws/eks/kubernetes_client.py index 2b0fc1978b5..2e7cd0a3e96 100644 --- a/metaflow/plugins/aws/eks/kubernetes_client.py +++ b/metaflow/plugins/aws/eks/kubernetes_client.py @@ -90,14 +90,11 @@ def create(self): ) # TODO(s) (savin) - # 1. Find a way to ensure that a pod is cleanly terminated automatically - # if the container fails to start properly (invalid docker image - # etc.) - # 2. Add support for GPUs. + # 1. Add support for GPUs. # A discerning eye would notice and question the choice of using the - # V1Job construct over the V1Pod construct given that we don't rely on - # any of the V1Job semantics. The only reasons at the moment are - + # V1Job construct over the V1Pod construct given that we don't rely much + # on any of the V1Job semantics. The major reasons at the moment are - # 1. It makes the Kubernetes UIs (Octant, Lens) a bit more easy on # the eyes, although even that can be questioned. # 2. AWS Step Functions, at the moment (Aug' 21) only supports @@ -105,7 +102,7 @@ def create(self): # API. When we ship the AWS Step Functions integration with EKS, # it will hopefully lessen our workload. # - # The current implementation assumes that there is only one unique Pod + # Note: This implementation ensures that there is only one unique Pod # (unique UID) per Metaflow task attempt. self._job = self._client.V1Job( api_version="batch/v1", @@ -257,6 +254,7 @@ def execute(self): return RunningJob( client=self._client, name=response["metadata"]["name"], + uid=response["metadata"]["uid"], namespace=response["metadata"]["namespace"], ) except self._client.rest.ApiException as e: @@ -347,17 +345,17 @@ class RunningJob(object): # # WIP... - def __init__(self, client, name, namespace): + JOB_ACTIVE = "job:active" + JOB_FAILED = "" + + def __init__(self, client, name, uid, namespace): self._client = client self._name = name + self._id = uid self._namespace = namespace - self._pod = None - - self.update() - # Get the V1Job id (controller for the pod) - # TODO (savin): Should the id be job id, job name, pod id or pod name? - self._id = self._pod["metadata"]["labels"]["controller-uid"] + self._job = self._fetch_job() + self._pod = self._fetch_pod() import atexit @@ -368,6 +366,18 @@ def __repr__(self): self.__class__.__name__, self._namespace, self._name ) + def _fetch_job(self): + try: + return ( + self._client.BatchV1Api() + .read_namespaced_job(name=self._name, namespace=self._namespace) + .to_dict() + ) + except self._client.rest.ApiException as e: + # TODO: Handle failures as well as the fact that a different + # process can delete the job. + raise e + def _fetch_pod(self): try: # TODO (savin): pods may not appear immediately or they may @@ -385,26 +395,10 @@ def _fetch_pod(self): # TODO: Handle failures raise e - def _fetch_job(self): - try: - return self._client.BatchV1Api().read_namespaced_job( - name=self._name, namespace=self._namespace - ) - except self._client.rest.ApiException as e: - # TODO: Handle failures - raise e - - def update(self): - self._pod = self._fetch_pod() - # print(self._pod) - # print( - # self._pod["status"].get("container_statuses", [{}])[0].get("state") - # ) - return self - def kill(self): # Terminating a Kubernetes job is a bit tricky. Issuing a - # `BatchV1Api.delete_namespaced_job` will also remove all traces of the # job object from the Kubernetes API server which may not be desirable. + # `BatchV1Api.delete_namespaced_job` will also remove all traces of the + # job object from the Kubernetes API server which may not be desirable. # This forces us to be a bit creative in terms of how we handle kill: # # 1. If the container is alive and kicking inside the pod, we simply @@ -425,7 +419,6 @@ def kill(self): # 3. If the pod object hasn't shown up yet, we set the parallelism to 0 # to preempt it. if not self.is_done: - # TODO (savin): Swap the check with if container is running. if self.is_running: # Case 1. from kubernetes.stream import stream @@ -435,10 +428,7 @@ def kill(self): # TODO (savin): stream opens a web-socket connection. It may # not be desirable to open multiple web-socket # connections frivolously (think killing a - # workflow during a for-each step). Given that - # we are only interested in a fire-and-forget - # request, we should look into how to avoid - # the ws. + # workflow during a for-each step). stream( api_instance().connect_get_namespaced_pod_exec, name=self._pod["metadata"]["name"], @@ -454,7 +444,7 @@ def kill(self): tty=False, ) except: - # Best effort. It's likely that this API call could also be + # Best effort. It's likely that this API call could be # blocked for the user. # TODO (savin): Forward the error to the user. # pass @@ -479,52 +469,129 @@ def kill(self): @property def id(self): + # TODO (savin): Should we use pod id instead? return self._id @property def is_done(self): def _done(): - return self._pod["status"]["phase"] in ("Succeeded", "Failed") + # Either the job succeeds or fails naturally or we may have + # forced the pod termination causing the job to still be in an + # active state but for all intents and purposes dead to us. + # + # This method relies exclusively on the state of V1Job object, + # since it's guaranteed to exist during the lifetime of the job. + + # TODO (savin): check for self._job + return ( + bool(self._job["status"].get("succeeded")) + or bool(self._job["status"].get("failed")) + or (self._job["spec"]["parallelism"] == 0) + ) if not _done(): - # if not done, check for newer status - self.update() + # If not done, check for newer status + self._job = self._fetch_job() return _done() @property def status(self): if not self.is_done: - # if not done, check for newer status (see the implementation of - # self.is_done) - pass - return self._pod["status"]["phase"] + # If not done, check for newer status + self._pod = self._fetch_pod() + # Success! + if bool(self._job["status"].get("succeeded")): + return "Job:Succeeded" + # Failure! + if bool(self._job["status"].get("failed")) or ( + self._job["spec"]["parallelism"] == 0 + ): + return "Job:Failed" + if bool(self._job["status"].get("active")): + msg = "Job:Active" + if self._pod: + msg += " Pod:%s" % self._pod["status"]["phase"].title() + # TODO (savin): parse Pod conditions + container_status = ( + self._pod["status"].get("container_statuses") or [None] + )[0] + if container_status: + # We have a single container inside the pod + status = {"status": "waiting"} + for k, v in container_status["state"].items(): + if v is not None: + status["status"] = k + status.update(v) + msg += " Container:%s" % status["status"].title() + reason = "" + if status.get("reason"): + reason = status["reason"] + if status.get("message"): + reason += ":%s" % status["message"] + if reason: + msg += " [%s]" % reason + # TODO (savin): This message should be shortened before release. + return msg + return "Job:Unknown" @property def has_succeeded(self): - return self.status == "Succeeded" + # Job is in a terminal state and the status is marked as succeeded + return self.is_done and bool(self._job["status"].get("succeeded")) @property def has_failed(self): - return self.status == "Failed" + # Job is in a terminal state and either the status is marked as failed + # or the Job is not allowed to launch any more pods + return self.is_done and ( + bool(self._job["status"].get("failed")) + or (self._job["spec"]["parallelism"] == 0) + ) @property def is_running(self): - return self.status == "Running" - - @property - def is_pending(self): - return self.status == "Pending" + # The container is running. This happens when the Pod's phase is running + if not self.is_done: + # If not done, check if pod has been assigned and is in Running + # phase + self._pod = self._fetch_pod() + return self._pod.get("status", {}).get("phase") == "Running" + return False @property - def container_status(self): - return ( - self._pod["status"].get("container_statuses", [{}])[0].get("state") - ) + def is_waiting(self): + return not self.is_done and not self.is_running @property def reason(self): - return "foo" + if self.is_done: + if self.has_succeeded: + return 0, None + # Best effort since Pod object can disappear on us at anytime + else: - @property - def status_code(self): - return 1 + def _done(): + return self._pod.get("status", {}).get("phase") in ( + "Succeeded", + "Failed", + ) + + if not _done(): + # If pod status is dirty, check for newer status + self._pod = self._fetch_pod() + if self._pod: + for k, v in ( + self._pod["status"] + .get("container_statuses", [{}])[0] + .get("state", {}) + .items() + ): + if v is not None: + return v.get("exit_code"), ": ".join( + filter( + None, + [v.get("reason"), v.get("message")], + ) + ) + + return None, None diff --git a/metaflow/plugins/aws/eks/kubernetes_decorator.py b/metaflow/plugins/aws/eks/kubernetes_decorator.py index 35e878473e9..80fdeae0102 100644 --- a/metaflow/plugins/aws/eks/kubernetes_decorator.py +++ b/metaflow/plugins/aws/eks/kubernetes_decorator.py @@ -71,7 +71,7 @@ def my_step(self): "secrets": None, # e.g., mysecret "node_selector": None, # e.g., kubernetes.io/os=linux "gpu": "0", - "shared_memory": None, + # "shared_memory": None, "name_space": None, } package_url = None From f8a4c4e3194663226a5f912b98354679aedbcc60 Mon Sep 17 00:00:00 2001 From: savin Date: Fri, 27 Aug 2021 23:07:15 -0700 Subject: [PATCH 20/32] minor fixups --- metaflow/plugins/aws/eks/kubernetes_cli.py | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/metaflow/plugins/aws/eks/kubernetes_cli.py b/metaflow/plugins/aws/eks/kubernetes_cli.py index 2cdb2d6df44..d6c44d54e5f 100644 --- a/metaflow/plugins/aws/eks/kubernetes_cli.py +++ b/metaflow/plugins/aws/eks/kubernetes_cli.py @@ -6,6 +6,7 @@ from metaflow import util from metaflow.exception import CommandException, METAFLOW_EXIT_DISALLOW_RETRY +from metaflow.metaflow_config import DATASTORE_LOCAL_DIR from .kubernetes import Kubernetes, KubernetesKilledException from ..aws_utils import sync_metadata_from_S3 @@ -13,7 +14,7 @@ # TODO(s): # 1. Compatibility for Metaflow-R (not a blocker for release). -# 2. +# 2. Add more CLI commands to manage Kubernetes objects. @click.group() @@ -180,6 +181,12 @@ def echo(msg, stream="stderr", job_id=None): step_args=" ".join(util.dict_to_cli_options(kwargs)), ) + def _sync_metadata(): + if ctx.obj.metadata.TYPE == 'local': + sync_metadata_from_S3(DATASTORE_LOCAL_DIR, + datastore_root, + retry_count) + try: kubernetes = Kubernetes( datastore=ctx.obj.datastore, @@ -215,7 +222,7 @@ def echo(msg, stream="stderr", job_id=None): # TODO: Make sure all errors pretty print nicely. traceback.print_exc() # print(str(e)) - sync_metadata_from_S3(ctx.obj.metadata, datastore_root, retry_count) + _sync_metadata() sys.exit(METAFLOW_EXIT_DISALLOW_RETRY) try: # Wait for the Kubernetes job to finish. @@ -223,6 +230,6 @@ def echo(msg, stream="stderr", job_id=None): except KubernetesKilledException: # Don't retry killed jobs. traceback.print_exc() - sync_metadata_from_S3(ctx.obj.metadata, datastore_root, retry_count) + _sync_metadata() sys.exit(METAFLOW_EXIT_DISALLOW_RETRY) - sync_metadata_from_S3(ctx.obj.metadata, datastore_root, retry_count) + _sync_metadata() From c784b16223dd6dec874479300ba9b8cc4d886aec Mon Sep 17 00:00:00 2001 From: Oleg Avdeev Date: Tue, 21 Sep 2021 18:11:52 -0700 Subject: [PATCH 21/32] better error handling for evicted pods (#711) --- metaflow/plugins/aws/eks/kubernetes_client.py | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/metaflow/plugins/aws/eks/kubernetes_client.py b/metaflow/plugins/aws/eks/kubernetes_client.py index 2e7cd0a3e96..56521720cce 100644 --- a/metaflow/plugins/aws/eks/kubernetes_client.py +++ b/metaflow/plugins/aws/eks/kubernetes_client.py @@ -580,8 +580,19 @@ def _done(): # If pod status is dirty, check for newer status self._pod = self._fetch_pod() if self._pod: + pod_status = self._pod["status"] + if pod_status.get("container_statuses") is None: + # We're done, but no container_statuses is set + # This can happen when the pod is evicted + return None, ": ".join( + filter( + None, + [pod_status.get("reason"), pod_status.get("message")], + ) + ) + for k, v in ( - self._pod["status"] + pod_status .get("container_statuses", [{}])[0] .get("state", {}) .items() From c8d5610de8060569c50e8c7d2de7b95e69091cf8 Mon Sep 17 00:00:00 2001 From: Oleg Avdeev Date: Thu, 7 Oct 2021 16:32:50 -0700 Subject: [PATCH 22/32] fixes for pod/job metadata race conditions (#704) --- metaflow/plugins/aws/eks/kubernetes.py | 4 +- metaflow/plugins/aws/eks/kubernetes_client.py | 258 +++++++++++------- 2 files changed, 165 insertions(+), 97 deletions(-) diff --git a/metaflow/plugins/aws/eks/kubernetes.py b/metaflow/plugins/aws/eks/kubernetes.py index f1acfb30fd4..8c2a945b0c0 100644 --- a/metaflow/plugins/aws/eks/kubernetes.py +++ b/metaflow/plugins/aws/eks/kubernetes.py @@ -358,9 +358,7 @@ def _print_available(tail, stream, should_persist=False): "%s. This could be a transient error. " "Use @retry to retry." % msg ) - elif not self._job.is_done: - # Kill the job if it is still running by throwing an exception. - raise KubernetesKilledException("Task failed!") + exit_code, _ = self._job.reason echo( "Task finished with exit code %s." % exit_code, diff --git a/metaflow/plugins/aws/eks/kubernetes_client.py b/metaflow/plugins/aws/eks/kubernetes_client.py index 56521720cce..e56c2f68de0 100644 --- a/metaflow/plugins/aws/eks/kubernetes_client.py +++ b/metaflow/plugins/aws/eks/kubernetes_client.py @@ -1,4 +1,5 @@ import os +import time try: unicode @@ -8,6 +9,8 @@ from metaflow.exception import MetaflowException +CLIENT_REFRESH_INTERVAL_SECONDS = 300 + class KubernetesJobException(MetaflowException): headline = "Kubernetes job error" @@ -28,6 +31,11 @@ def __init__(self): "Could not import module 'kubernetes'. Install kubernetes " "Python package (https://pypi.org/project/kubernetes/) first." ) + self._refresh_client() + + def _refresh_client(self): + from kubernetes import client, config + if os.getenv("KUBERNETES_SERVICE_HOST"): # We’re inside a pod, authenticate via ServiceAccount assigned to us config.load_incluster_config() @@ -40,14 +48,24 @@ def __init__(self): # good enough for the initial rollout. config.load_kube_config() self._client = client + self._client_refresh_timestamp = time.time() def job(self, **kwargs): - return KubernetesJob(self._client, **kwargs) + return KubernetesJob(self, **kwargs) + + def get(self): + if ( + time.time() - self._client_refresh_timestamp + > CLIENT_REFRESH_INTERVAL_SECONDS + ): + self._refresh_client() + + return self._client class KubernetesJob(object): - def __init__(self, client, **kwargs): - self._client = client + def __init__(self, client_wrapper, **kwargs): + self._client_wrapper = client_wrapper self._kwargs = kwargs # Kubernetes namespace defaults to `default` @@ -104,10 +122,11 @@ def create(self): # # Note: This implementation ensures that there is only one unique Pod # (unique UID) per Metaflow task attempt. - self._job = self._client.V1Job( + client = self._client_wrapper.get() + self._job = client.V1Job( api_version="batch/v1", kind="Job", - metadata=self._client.V1ObjectMeta( + metadata=client.V1ObjectMeta( # Annotations are for humans annotations=self._kwargs.get("annotations", {}), # While labels are for Kubernetes @@ -115,7 +134,7 @@ def create(self): name=self._kwargs["name"], # Unique within the namespace namespace=self._kwargs["namespace"], # Defaults to `default` ), - spec=self._client.V1JobSpec( + spec=client.V1JobSpec( # Retries are handled by Metaflow when it is responsible for # executing the flow. The responsibility is moved to Kubernetes # when AWS Step Functions / Argo are responsible for the @@ -128,18 +147,16 @@ def create(self): * 60 * 60 # Remove job after a week. TODO (savin): Make this * 24, # configurable - template=self._client.V1PodTemplateSpec( - metadata=self._client.V1ObjectMeta( + template=client.V1PodTemplateSpec( + metadata=client.V1ObjectMeta( annotations=self._kwargs.get("annotations", {}), labels=self._kwargs.get("labels", {}), name=self._kwargs["name"], namespace=self._kwargs["namespace"], ), - spec=self._client.V1PodSpec( + spec=client.V1PodSpec( # Timeout is set on the pod and not the job (important!) - active_deadline_seconds=self._kwargs[ - "timeout_in_seconds" - ], + active_deadline_seconds=self._kwargs["timeout_in_seconds"], # TODO (savin): Enable affinities for GPU scheduling. # This requires some thought around the # UX since specifying affinities can get @@ -148,10 +165,10 @@ def create(self): # roll out. # affinity=?, containers=[ - self._client.V1Container( + client.V1Container( command=self._kwargs["command"], env=[ - self._client.V1EnvVar(name=k, value=str(v)) + client.V1EnvVar(name=k, value=str(v)) for k, v in self._kwargs.get( "environment_variables", {} ).items() @@ -163,10 +180,10 @@ def create(self): # TODO: Figure out a way to make job # metadata visible within the container + [ - self._client.V1EnvVar( + client.V1EnvVar( name=k, - value_from=self._client.V1EnvVarSource( - field_ref=self._client.V1ObjectFieldSelector( + value_from=client.V1EnvVarSource( + field_ref=client.V1ObjectFieldSelector( field_path=str(v) ) ), @@ -178,20 +195,17 @@ def create(self): }.items() ], env_from=[ - self._client.V1EnvFromSource( - secret_ref=self._client.V1SecretEnvSource( - name=str(k) - ) + client.V1EnvFromSource( + secret_ref=client.V1SecretEnvSource(name=str(k)) ) for k in self._kwargs.get("secrets", []) ], image=self._kwargs["image"], name=self._kwargs["name"], - resources=self._client.V1ResourceRequirements( + resources=client.V1ResourceRequirements( requests={ "cpu": str(self._kwargs["cpu"]), - "memory": "%sM" - % str(self._kwargs["memory"]), + "memory": "%sM" % str(self._kwargs["memory"]), "ephemeral-storage": "%sM" % str(self._kwargs["disk"]), } @@ -239,25 +253,26 @@ def create(self): return self def execute(self): + client = self._client_wrapper.get() try: # TODO (savin): Make job submission back-pressure aware. Currently # there doesn't seem to be a kubernetes-native way to # achieve the guarantees that we are seeking. # Hopefully, we will be able to get creative soon. response = ( - self._client.BatchV1Api() + client.BatchV1Api() .create_namespaced_job( body=self._job, namespace=self._kwargs["namespace"] ) .to_dict() ) return RunningJob( - client=self._client, + client_wrapper=self._client_wrapper, name=response["metadata"]["name"], uid=response["metadata"]["uid"], namespace=response["metadata"]["namespace"], ) - except self._client.rest.ApiException as e: + except client.rest.ApiException as e: raise KubernetesJobException( "Unable to launch Kubernetes job.\n %s" % str(e) ) @@ -293,9 +308,7 @@ def environment_variable(self, name, value): return self def label(self, name, value): - self._kwargs["labels"] = dict( - self._kwargs.get("labels", {}), **{name: value} - ) + self._kwargs["labels"] = dict(self._kwargs.get("labels", {}), **{name: value}) return self def annotation(self, name, value): @@ -309,6 +322,22 @@ class RunningJob(object): # State Machine implementation for the lifecycle behavior documented in # https://kubernetes.io/docs/concepts/workloads/pods/pod-lifecycle/ + # + # This object encapsulates *both* V1Job and V1Pod. It simplifies the status + # to "running" and "done" (failed/succeeded) state. Note that V1Job and V1Pod + # status fields are not guaranteed to be always in sync due to the way job + # controller works. + # + # For example, for a successful job, RunningJob states and their corresponding + # K8S object states look like this: + # + # | V1JobStatus.active | V1JobStatus.succeeded | V1PodStatus.phase | RunningJob.is_running | RunningJob.is_done | + # |--------------------|-----------------------|-------------------|-----------------------|--------------------| + # | 0 | 0 | N/A | False | False | + # | 0 | 0 | Pending | False | False | + # | 1 | 0 | Running | True | False | + # | 1 | 0 | Succeeded | True | True | + # | 0 | 1 | Succeeded | False | True | # To ascertain the status of V1Job, we peer into the lifecycle status of # the pod it is responsible for executing. Unfortunately, the `phase` @@ -348,8 +377,8 @@ class RunningJob(object): JOB_ACTIVE = "job:active" JOB_FAILED = "" - def __init__(self, client, name, uid, namespace): - self._client = client + def __init__(self, client_wrapper, name, uid, namespace): + self._client_wrapper = client_wrapper self._name = name self._id = uid self._namespace = namespace @@ -367,33 +396,35 @@ def __repr__(self): ) def _fetch_job(self): + client = self._client_wrapper.get() try: return ( - self._client.BatchV1Api() + client.BatchV1Api() .read_namespaced_job(name=self._name, namespace=self._namespace) .to_dict() ) - except self._client.rest.ApiException as e: + except client.rest.ApiException as e: # TODO: Handle failures as well as the fact that a different # process can delete the job. raise e def _fetch_pod(self): - try: - # TODO (savin): pods may not appear immediately or they may - # disappear - return ( - self._client.CoreV1Api() - .list_namespaced_pod( - namespace=self._namespace, - label_selector="job-name={}".format(self._name), - ) - .to_dict()["items"] - or [None] - )[0] - except self._client.rest.ApiException as e: - # TODO: Handle failures - raise e + """Fetch pod metadata. May return None if pod does not exist.""" + client = self._client_wrapper.get() + + pods = ( + client.CoreV1Api() + .list_namespaced_pod( + namespace=self._namespace, + label_selector="job-name={}".format(self._name), + ) + .to_dict()["items"] + ) + + if pods: + return pods[0] + else: + return None def kill(self): # Terminating a Kubernetes job is a bit tricky. Issuing a @@ -418,12 +449,18 @@ def kill(self): # terminate the pod without deleting the object. # 3. If the pod object hasn't shown up yet, we set the parallelism to 0 # to preempt it. - if not self.is_done: - if self.is_running: + client = self._client_wrapper.get() + if not self._check_is_done(): + if self._check_is_running(): + + # Unless there is a bug in the code, self._pod cannot be None + # if we're in "running" state. + assert self._pod is not None + # Case 1. from kubernetes.stream import stream - api_instance = self._client.CoreV1Api + api_instance = client.CoreV1Api try: # TODO (savin): stream opens a web-socket connection. It may # not be desirable to open multiple web-socket @@ -454,7 +491,7 @@ def kill(self): try: # TODO (savin): Also patch job annotation to reflect this # action. - self._client.BatchV1Api().patch_namespaced_job( + client.BatchV1Api().patch_namespaced_job( name=self._name, namespace=self._namespace, field_manager="metaflow", @@ -472,15 +509,11 @@ def id(self): # TODO (savin): Should we use pod id instead? return self._id - @property - def is_done(self): - def _done(): + def _check_is_done(self): + def _job_done(): # Either the job succeeds or fails naturally or we may have # forced the pod termination causing the job to still be in an # active state but for all intents and purposes dead to us. - # - # This method relies exclusively on the state of V1Job object, - # since it's guaranteed to exist during the lifetime of the job. # TODO (savin): check for self._job return ( @@ -489,15 +522,23 @@ def _done(): or (self._job["spec"]["parallelism"] == 0) ) - if not _done(): + if not _job_done(): # If not done, check for newer status self._job = self._fetch_job() - return _done() + if _job_done(): + return True + else: + # It is possible for the job metadata to not be updated yet, but the + # Pod has already succeeded or failed. + self._pod = self._fetch_pod() + if self._pod and (self._pod["status"]["phase"] in ("Succeeded", "Failed")): + return True + else: + return False - @property - def status(self): - if not self.is_done: - # If not done, check for newer status + def _get_status(self): + if not self._check_is_done(): + # If not done, check for newer pod status self._pod = self._fetch_pod() # Success! if bool(self._job["status"].get("succeeded")): @@ -534,38 +575,48 @@ def status(self): return msg return "Job:Unknown" - @property - def has_succeeded(self): + def _check_has_succeeded(self): # Job is in a terminal state and the status is marked as succeeded - return self.is_done and bool(self._job["status"].get("succeeded")) + if self._check_is_done(): + if bool(self._job["status"].get("succeeded")) or ( + self._pod and self._pod["status"]["phase"] == "Succeeded" + ): + return True + else: + return False + else: + return False - @property - def has_failed(self): + def _check_has_failed(self): # Job is in a terminal state and either the status is marked as failed # or the Job is not allowed to launch any more pods - return self.is_done and ( - bool(self._job["status"].get("failed")) - or (self._job["spec"]["parallelism"] == 0) - ) - @property - def is_running(self): - # The container is running. This happens when the Pod's phase is running - if not self.is_done: + if self._check_is_done(): + if ( + bool(self._job["status"].get("failed")) + or (self._job["spec"]["parallelism"] == 0) + or (self._pod and self._pod["status"]["phase"] == "Failed") + ): + return True + else: + return False + else: + return False + + def _check_is_running(self): + # Returns true if the container is running. + if not self._check_is_done(): # If not done, check if pod has been assigned and is in Running # phase - self._pod = self._fetch_pod() - return self._pod.get("status", {}).get("phase") == "Running" + if self._pod is None: + return False + pod_phase = self._pod.get("status", {}).get("phase") + return pod_phase == "Running" return False - @property - def is_waiting(self): - return not self.is_done and not self.is_running - - @property - def reason(self): - if self.is_done: - if self.has_succeeded: + def _get_done_reason(self): + if self._check_is_done(): + if self._check_has_succeeded(): return 0, None # Best effort since Pod object can disappear on us at anytime else: @@ -585,15 +636,14 @@ def _done(): # We're done, but no container_statuses is set # This can happen when the pod is evicted return None, ": ".join( - filter( - None, - [pod_status.get("reason"), pod_status.get("message")], - ) + filter( + None, + [pod_status.get("reason"), pod_status.get("message")], ) + ) for k, v in ( - pod_status - .get("container_statuses", [{}])[0] + pod_status.get("container_statuses", [{}])[0] .get("state", {}) .items() ): @@ -606,3 +656,23 @@ def _done(): ) return None, None + + @property + def is_done(self): + return self._check_is_done() + + @property + def has_failed(self): + return self._check_has_failed() + + @property + def is_running(self): + return self._check_is_running() + + @property + def reason(self): + return self._get_done_reason() + + @property + def status(self): + return self._get_status() From 8ab13365c9ce02ab33c24d1dd5ee0e182217e804 Mon Sep 17 00:00:00 2001 From: Oleg Avdeev Date: Thu, 7 Oct 2021 17:17:12 -0700 Subject: [PATCH 23/32] K8S: label value sanitizer (#719) --- metaflow/plugins/aws/eks/kubernetes.py | 94 ++++++++++++++++++------ test/unit/test_k8s_job_name_sanitizer.py | 26 +++++++ test/unit/test_k8s_label_sanitizer.py | 28 +++++++ 3 files changed, 125 insertions(+), 23 deletions(-) create mode 100644 test/unit/test_k8s_job_name_sanitizer.py create mode 100644 test/unit/test_k8s_label_sanitizer.py diff --git a/metaflow/plugins/aws/eks/kubernetes.py b/metaflow/plugins/aws/eks/kubernetes.py index 8c2a945b0c0..c193430f6a4 100644 --- a/metaflow/plugins/aws/eks/kubernetes.py +++ b/metaflow/plugins/aws/eks/kubernetes.py @@ -5,6 +5,7 @@ import shlex import time import re +import hashlib from metaflow import util from metaflow.datastore.util.s3tail import S3Tail @@ -44,6 +45,62 @@ class KubernetesKilledException(MetaflowException): headline = "Kubernetes Batch job killed" +def generate_rfc1123_name(flow_name, + run_id, + step_name, + task_id, + attempt +): + """ + Generate RFC 1123 compatible name. Specifically, the format is: + [*[]] + + The generated name consists from a human-readable prefix, derived from + flow/step/task/attempt, and a hash suffux. + """ + long_name = "-".join( + [ + flow_name, + run_id, + step_name, + task_id, + attempt, + ] + ) + hash = hashlib.sha256(long_name.encode('utf-8')).hexdigest() + + if long_name.startswith('_'): + # RFC 1123 names can't start with hyphen so slap an extra prefix on it + sanitized_long_name = 'u' + long_name.replace('_', '-').lower() + else: + sanitized_long_name = long_name.replace('_', '-').lower() + + # the name has to be under 63 chars total + return sanitized_long_name[:57] + '-' + hash[:5] + + +LABEL_VALUE_REGEX = re.compile(r'^[a-zA-Z0-9]([a-zA-Z0-9\-\_\.]{0,61}[a-zA-Z0-9])?$') + + +def sanitize_label_value(val): + # Label sanitization: if the value can be used as is, return it as is. + # If it can't, sanitize and add a suffix based on hash of the original + # value, replace invalid chars and truncate. + # + # The idea here is that even if there are non-allowed chars in the same + # position, this function will likely return distinct values, so you can + # still filter on those. For example, "alice$" and "alice&" will be + # sanitized into different values "alice_b3f201" and "alice_2a6f13". + if val == '' or LABEL_VALUE_REGEX.match(val): + return val + hash = hashlib.sha256(val.encode('utf-8')).hexdigest() + + # Replace invalid chars with dots, and if the first char is + # non-alphahanumeric, replace it with 'u' to make it valid + sanitized_val = re.sub('^[^A-Z0-9a-z]', 'u', re.sub(r"[^A-Za-z0-9.\-_]", "_", val)) + return sanitized_val[:57] + '-' + hash[:5] + + class Kubernetes(object): def __init__( self, @@ -150,15 +207,13 @@ def create_job( # attempt_id while submitting the job to the Kubernetes cluster. If # that is indeed the case, we can rely on Kubernetes to generate a name # for us. - job_name = "-".join( - [ - self._flow_name, - self._run_id, - self._step_name, - self._task_id, - self._attempt, - ] - ).lower() + job_name = generate_rfc1123_name( + self._flow_name, + self._run_id, + self._step_name, + self._task_id, + self._attempt, + ) job = ( KubernetesClient() @@ -205,11 +260,11 @@ def create_job( .environment_variable("METAFLOW_DEFAULT_METADATA", DEFAULT_METADATA) .environment_variable("METAFLOW_KUBERNETES_WORKLOAD", 1) .label("app", "metaflow") - .label("metaflow/flow_name", self._flow_name) - .label("metaflow/run_id", self._run_id) - .label("metaflow/step_name", self._step_name) - .label("metaflow/task_id", self._task_id) - .label("metaflow/attempt", self._attempt) + .label("metaflow/flow_name", sanitize_label_value(self._flow_name)) + .label("metaflow/run_id", sanitize_label_value(self._run_id)) + .label("metaflow/step_name", sanitize_label_value(self._step_name)) + .label("metaflow/task_id", sanitize_label_value(self._task_id)) + .label("metaflow/attempt", sanitize_label_value(self._attempt)) ) # Skip setting METAFLOW_DATASTORE_SYSROOT_LOCAL because metadata sync @@ -229,19 +284,12 @@ def create_job( # introducing them here. job.label("app.kubernetes.io/name", "metaflow-task").label( "app.kubernetes.io/part-of", "metaflow" - ).label("app.kubernetes.io/created-by", user) + ).label("app.kubernetes.io/created-by", sanitize_label_value(user)) # Add Metaflow system tags as labels as well! - # - # TODO 1. Label values must be an empty string or consist of - # alphanumeric characters, '-', '_' or '.', and must start and - # end with an alphanumeric character. Fix the simple regex - # match below. for sys_tag in self._metadata.sticky_sys_tags: job.label( "metaflow/%s" % sys_tag[: sys_tag.index(":")], - re.sub( - "[^A-Za-z0-9.-_]", ".", sys_tag[sys_tag.index(":") + 1 :] - ), + sanitize_label_value(sys_tag[sys_tag.index(":") + 1 :]) ) # TODO: Add annotations based on https://kubernetes.io/blog/2021/04/20/annotating-k8s-for-humans/ diff --git a/test/unit/test_k8s_job_name_sanitizer.py b/test/unit/test_k8s_job_name_sanitizer.py new file mode 100644 index 00000000000..e019a47bea3 --- /dev/null +++ b/test/unit/test_k8s_job_name_sanitizer.py @@ -0,0 +1,26 @@ +import re +from metaflow.plugins.aws.eks.kubernetes import generate_rfc1123_name + +rfc1123 = re.compile(r'^[a-zA-Z0-9]([a-zA-Z0-9\-]{0,61}[a-zA-Z0-9])?$') + +def test_job_name_santitizer(): + # Basic name + assert rfc1123.match(generate_rfc1123_name('HelloFlow', '1', 'end', '321', '1')) + + # Step name ends with _ + assert rfc1123.match(generate_rfc1123_name('HelloFlow', '1', '_end', '321', '1')) + + # Step name starts and ends with _ + assert rfc1123.match(generate_rfc1123_name('HelloFlow', '1', '_end_', '321', '1')) + + # Flow name ends with _ + assert rfc1123.match(generate_rfc1123_name('HelloFlow_', '1', 'end', '321', '1')) + + # Same flow name, different case must produce different job names + assert generate_rfc1123_name('Helloflow', '1', 'end', '321', '1') != generate_rfc1123_name('HelloFlow', '1', 'end', '321', '1') + + # Very long step name should be fine + assert rfc1123.match(generate_rfc1123_name('Helloflow', '1', 'end'*50, '321', '1')) + + # Very long run id should be fine too + assert rfc1123.match(generate_rfc1123_name('Helloflow', '1'*100, 'end', '321', '1')) \ No newline at end of file diff --git a/test/unit/test_k8s_label_sanitizer.py b/test/unit/test_k8s_label_sanitizer.py new file mode 100644 index 00000000000..6fcfbd5553f --- /dev/null +++ b/test/unit/test_k8s_label_sanitizer.py @@ -0,0 +1,28 @@ +import re +from metaflow.plugins.aws.eks.kubernetes import sanitize_label_value, LABEL_VALUE_REGEX + + +def test_label_value_santitizer(): + assert LABEL_VALUE_REGEX.match(sanitize_label_value('HelloFlow')) + + # The value is too long + assert LABEL_VALUE_REGEX.match(sanitize_label_value('a' * 1000)) + + # Different long values should still not be equal after sanitization + assert sanitize_label_value('a' * 1000) != sanitize_label_value('a' * 1001) + assert sanitize_label_value('-' * 1000) != sanitize_label_value('-' * 1001) + + # Different long values should still not be equal after sanitization + assert sanitize_label_value('alice!') != sanitize_label_value('alice?') + + # ends with dash + assert LABEL_VALUE_REGEX.match(sanitize_label_value('HelloFlow-')) + + # non-ascii + assert LABEL_VALUE_REGEX.match(sanitize_label_value('метафлоу')) + + # different only in case + assert sanitize_label_value('Alice') != sanitize_label_value('alice') + + # spaces + assert LABEL_VALUE_REGEX.match(sanitize_label_value('Meta flow')) \ No newline at end of file From fb73234be9b8362ea43f353193ec8b4186ba3e4a Mon Sep 17 00:00:00 2001 From: Oleg Avdeev Date: Tue, 12 Oct 2021 08:54:48 -0700 Subject: [PATCH 24/32] rename name_space to namespace for k8s plugin (#750) --- metaflow/plugins/aws/eks/kubernetes_cli.py | 8 ++++---- metaflow/plugins/aws/eks/kubernetes_decorator.py | 11 +++++++++-- 2 files changed, 13 insertions(+), 6 deletions(-) diff --git a/metaflow/plugins/aws/eks/kubernetes_cli.py b/metaflow/plugins/aws/eks/kubernetes_cli.py index d6c44d54e5f..4856f6d4863 100644 --- a/metaflow/plugins/aws/eks/kubernetes_cli.py +++ b/metaflow/plugins/aws/eks/kubernetes_cli.py @@ -61,9 +61,9 @@ def kubernetes(): ) @click.option( # Note that ideally we would have liked to use `namespace` rather than - # `name-space` but unfortunately, `namespace` is already reserved for + # `k8s-namespace` but unfortunately, `namespace` is already reserved for # Metaflow namespaces. - "--name-space", + "--k8s-namespace", default=None, help="Namespace for Kubernetes job on Amazon EKS.", ) @@ -109,7 +109,7 @@ def step( service_account=None, secrets=None, node_selector=None, - name_space=None, + k8s_namespace=None, cpu=None, gpu=None, disk=None, @@ -210,7 +210,7 @@ def _sync_metadata(): service_account=service_account, secrets=secrets, node_selector=node_selector, - namespace=name_space, + namespace=k8s_namespace, cpu=cpu, gpu=gpu, disk=disk, diff --git a/metaflow/plugins/aws/eks/kubernetes_decorator.py b/metaflow/plugins/aws/eks/kubernetes_decorator.py index 80fdeae0102..813aab054a4 100644 --- a/metaflow/plugins/aws/eks/kubernetes_decorator.py +++ b/metaflow/plugins/aws/eks/kubernetes_decorator.py @@ -72,7 +72,7 @@ def my_step(self): "node_selector": None, # e.g., kubernetes.io/os=linux "gpu": "0", # "shared_memory": None, - "name_space": None, + "namespace": None, } package_url = None package_sha = None @@ -180,7 +180,14 @@ def runtime_step_cli( cli_args.commands = ["kubernetes", "step"] cli_args.command_args.append(self.package_sha) cli_args.command_args.append(self.package_url) - cli_args.command_options.update(self.attributes) + + # --namespace is used to specify Metaflow namespace (different + # concept from k8s namespace). + for k,v in self.attributes.items(): + if k == 'namespace': + cli_args.command_options['k8s_namespace'] = v + else: + cli_args.command_options['namespace'] = v cli_args.command_options["run-time-limit"] = self.run_time_limit cli_args.entrypoint[0] = sys.executable From 987a6ee2b81982ca420ae4748438e57fd8464fe0 Mon Sep 17 00:00:00 2001 From: Oleg Avdeev Date: Tue, 12 Oct 2021 09:23:50 -0700 Subject: [PATCH 25/32] fix k8s attribute handling bug (#753) --- metaflow/plugins/aws/eks/kubernetes_decorator.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/metaflow/plugins/aws/eks/kubernetes_decorator.py b/metaflow/plugins/aws/eks/kubernetes_decorator.py index 813aab054a4..2f06b5fe0fc 100644 --- a/metaflow/plugins/aws/eks/kubernetes_decorator.py +++ b/metaflow/plugins/aws/eks/kubernetes_decorator.py @@ -187,7 +187,7 @@ def runtime_step_cli( if k == 'namespace': cli_args.command_options['k8s_namespace'] = v else: - cli_args.command_options['namespace'] = v + cli_args.command_options[k] = v cli_args.command_options["run-time-limit"] = self.run_time_limit cli_args.entrypoint[0] = sys.executable From f9042d814a587b7f6c406d8319c530f311b14190 Mon Sep 17 00:00:00 2001 From: Oleg Avdeev Date: Tue, 12 Oct 2021 10:25:39 -0700 Subject: [PATCH 26/32] tweak k8s test resources (to run on kind) (#754) --- test/core/contexts.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/core/contexts.json b/test/core/contexts.json index 562303317d8..7093c85d278 100644 --- a/test/core/contexts.json +++ b/test/core/contexts.json @@ -100,7 +100,7 @@ "--event-logger=nullSidecarLogger", "--no-pylint", "--quiet", - "--with=kubernetes", + "--with=kubernetes:memory=256,disk=1024", "--datastore=s3" ], "env": { From f58ae5849ad866a87cd4a80acb99272342b811b0 Mon Sep 17 00:00:00 2001 From: Oleg Avdeev Date: Wed, 13 Oct 2021 10:44:16 -0700 Subject: [PATCH 27/32] add k8s api retries (#756) --- metaflow/plugins/aws/eks/kubernetes_client.py | 38 +++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/metaflow/plugins/aws/eks/kubernetes_client.py b/metaflow/plugins/aws/eks/kubernetes_client.py index e56c2f68de0..e8fb5d22e7a 100644 --- a/metaflow/plugins/aws/eks/kubernetes_client.py +++ b/metaflow/plugins/aws/eks/kubernetes_client.py @@ -1,5 +1,8 @@ import os import time +import math +import random + try: unicode @@ -16,6 +19,39 @@ class KubernetesJobException(MetaflowException): headline = "Kubernetes job error" +# Implements truncated exponential backoff from https://cloud.google.com/storage/docs/retry-strategy#exponential-backoff +def k8s_retry(deadline_seconds=60, max_backoff=32): + def decorator(function): + from functools import wraps + + @wraps(function) + def wrapper(*args, **kwargs): + from kubernetes import client + + deadline = time.time() + deadline_seconds + retry_number = 0 + + while True: + try: + result = function(*args, **kwargs) + return result + except client.rest.ApiException as e: + if e.status == 500: + current_t = time.time() + backoff_delay = min(math.pow(2, retry_number) + random.random(), max_backoff) + if current_t + backoff_delay < deadline: + time.sleep(backoff_delay) + retry_number += 1 + continue # retry again + else: + raise + else: + raise + + return wrapper + return decorator + + class KubernetesClient(object): def __init__(self): # TODO: Look into removing the usage of Kubernetes Python SDK @@ -395,6 +431,7 @@ def __repr__(self): self.__class__.__name__, self._namespace, self._name ) + @k8s_retry() def _fetch_job(self): client = self._client_wrapper.get() try: @@ -408,6 +445,7 @@ def _fetch_job(self): # process can delete the job. raise e + @k8s_retry() def _fetch_pod(self): """Fetch pod metadata. May return None if pod does not exist.""" client = self._client_wrapper.get() From 135c9a6dcd19a4e20e0565604938d75fa00c7906 Mon Sep 17 00:00:00 2001 From: savin Date: Wed, 13 Oct 2021 11:30:23 -0700 Subject: [PATCH 28/32] update done marker --- metaflow/task.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/metaflow/task.py b/metaflow/task.py index b49bf77b0aa..b98a2c8c0ee 100644 --- a/metaflow/task.py +++ b/metaflow/task.py @@ -496,10 +496,6 @@ def run_step(self, output.save_metadata({'task_end': {}}) output.persist(self.flow) - # this writes a success marker indicating that the - # "transaction" is done - output.done() - # final decorator hook: The task results are now # queryable through the client API / datastore for deco in decorators: @@ -510,6 +506,10 @@ def run_step(self, retry_count, max_user_code_retries) + # this writes a success marker indicating that the + # "transaction" is done + output.done() + # terminate side cars logger.terminate() self.metadata.stop_heartbeat() From 92c8d259bf99a1eb4b2b48e3304efc56034b4618 Mon Sep 17 00:00:00 2001 From: Roman Kindruk <36699371+sappier@users.noreply.github.com> Date: Wed, 13 Oct 2021 22:43:40 +0200 Subject: [PATCH 29/32] Use linux binaries in @conda when run in k8s (#758) Conda environment should pack linux python binary when run on MacOS to avoid an error metaflow_PlayListFlow_osx-64_179c56284704ca8e53622f848a3df27cdd1f4327/bin/python: cannot execute binary file: Exec format error --- metaflow/plugins/conda/conda_step_decorator.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/metaflow/plugins/conda/conda_step_decorator.py b/metaflow/plugins/conda/conda_step_decorator.py index cbbbeec6bbb..10a9b9894f2 100644 --- a/metaflow/plugins/conda/conda_step_decorator.py +++ b/metaflow/plugins/conda/conda_step_decorator.py @@ -193,13 +193,13 @@ def _disable_safety_checks(self, decos): # a macOS. This is needed because of gotchas around inconsistently # case-(in)sensitive filesystems for macOS and linux. for deco in decos: - if deco.name == 'batch' and platform.system() == 'Darwin': + if deco.name in ('batch', 'kubernetes') and platform.system() == 'Darwin': return True return False def _architecture(self, decos): for deco in decos: - if deco.name == 'batch': + if deco.name in ('batch', 'kubernetes'): # force conda resolution for linux-64 architectures return 'linux-64' bit = '32' @@ -306,7 +306,9 @@ def runtime_step_cli(self, retry_count, max_user_code_retries, ubf_context): - if self.is_enabled(ubf_context) and 'batch' not in cli_args.commands: + no_batch = 'batch' not in cli_args.commands + no_kubernetes = 'kubernetes' not in cli_args.commands + if self.is_enabled(ubf_context) and no_batch and no_kubernetes: python_path = self.metaflow_home if self.addl_paths is not None: addl_paths = os.pathsep.join(self.addl_paths) From fb5f55838d3c027454b8bd2aeff13127e2ca3cf0 Mon Sep 17 00:00:00 2001 From: savin Date: Fri, 15 Oct 2021 10:31:33 -0700 Subject: [PATCH 30/32] fix comment --- metaflow/plugins/aws/batch/batch_decorator.py | 4 ---- 1 file changed, 4 deletions(-) diff --git a/metaflow/plugins/aws/batch/batch_decorator.py b/metaflow/plugins/aws/batch/batch_decorator.py index b3c988b0e90..02fa959b482 100644 --- a/metaflow/plugins/aws/batch/batch_decorator.py +++ b/metaflow/plugins/aws/batch/batch_decorator.py @@ -143,11 +143,7 @@ def step_init(self, for deco in decos: if isinstance(deco, ResourcesDecorator): for k, v in deco.attributes.items(): -<<<<<<< HEAD - # We use the larger of @resources and @k8s attributes -======= # We use the larger of @resources and @batch attributes ->>>>>>> master # TODO: Fix https://github.com/Netflix/metaflow/issues/467 my_val = self.attributes.get(k) if not (my_val is None and v is None): From 2eb6ba93db2a0c93023997105b95bc0a1e5a2aff Mon Sep 17 00:00:00 2001 From: savin Date: Fri, 15 Oct 2021 10:58:52 -0700 Subject: [PATCH 31/32] fix merge conflict --- metaflow/plugins/aws/batch/batch_decorator.py | 41 ------------------- 1 file changed, 41 deletions(-) diff --git a/metaflow/plugins/aws/batch/batch_decorator.py b/metaflow/plugins/aws/batch/batch_decorator.py index 02fa959b482..2ffb77c1144 100644 --- a/metaflow/plugins/aws/batch/batch_decorator.py +++ b/metaflow/plugins/aws/batch/batch_decorator.py @@ -208,47 +208,6 @@ def task_pre_step(self, inputs): self.metadata = metadata self.task_datastore = task_datastore -<<<<<<< HEAD - - # task_pre_step may run locally if fallback is activated for @catch - # decorator. In that scenario, we skip collecting AWS Batch execution - # metadata. A rudimentary way to detect non-local execution is to - # check for the existence of AWS_BATCH_JOB_ID environment variable. - - if 'AWS_BATCH_JOB_ID' in os.environ: - meta = {} - meta['aws-batch-job-id'] = os.environ['AWS_BATCH_JOB_ID'] - meta['aws-batch-job-attempt'] = os.environ['AWS_BATCH_JOB_ATTEMPT'] - meta['aws-batch-ce-name'] = os.environ['AWS_BATCH_CE_NAME'] - meta['aws-batch-jq-name'] = os.environ['AWS_BATCH_JQ_NAME'] - meta['aws-batch-execution-env'] = os.environ['AWS_EXECUTION_ENV'] - - - # Capture AWS Logs metadata. This is best effort only since - # only V4 of the metadata uri for the ECS container hosts this - # information and it is quite likely that not all consumers of - # Metaflow would be running the container agent compatible with - # version V4. - # https://docs.aws.amazon.com/AmazonECS/latest/developerguide/task-metadata-endpoint.html - try: - logs_meta = requests.get( - url=os.environ['ECS_CONTAINER_METADATA_URI_V4']) \ - .json() \ - .get('LogOptions', {}) - meta['aws-batch-awslogs-group'] = logs_meta.get('awslogs-group') - meta['aws-batch-awslogs-region'] = logs_meta.get('awslogs-region') - meta['aws-batch-awslogs-stream'] = logs_meta.get('awslogs-stream') - except: - pass - - entries = [MetaDatum( - field=k, value=v, type=k, tags=["attempt_id:{0}".format(retry_count)]) - for k, v in meta.items()] - # Register book-keeping metadata for debugging. - metadata.register_metadata(run_id, step_name, task_id, entries) - - self._save_logs_sidecar = SidecarSubProcess('save_logs_periodically') -======= # task_pre_step may run locally if fallback is activated for @catch # decorator. In that scenario, we skip collecting AWS Batch execution From 2bf5303b85c65191c4691d4f45522005458b7dd0 Mon Sep 17 00:00:00 2001 From: savin Date: Fri, 15 Oct 2021 15:24:49 -0700 Subject: [PATCH 32/32] update char --- metaflow/plugins/aws/eks/kubernetes_client.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/metaflow/plugins/aws/eks/kubernetes_client.py b/metaflow/plugins/aws/eks/kubernetes_client.py index e8fb5d22e7a..b0c143328c0 100644 --- a/metaflow/plugins/aws/eks/kubernetes_client.py +++ b/metaflow/plugins/aws/eks/kubernetes_client.py @@ -73,7 +73,7 @@ def _refresh_client(self): from kubernetes import client, config if os.getenv("KUBERNETES_SERVICE_HOST"): - # We’re inside a pod, authenticate via ServiceAccount assigned to us + # We are inside a pod, authenticate via ServiceAccount assigned to us config.load_incluster_config() else: # Use kubeconfig, likely $HOME/.kube/config