From commits-return-26635-archive-asf-public=cust-asf.ponee.io@airflow.incubator.apache.org Wed Oct 31 11:45:19 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id 6CF4C180668 for ; Wed, 31 Oct 2018 11:45:18 +0100 (CET) Received: (qmail 14387 invoked by uid 500); 31 Oct 2018 10:45:17 -0000 Mailing-List: contact commits-help@airflow.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@airflow.incubator.apache.org Delivered-To: mailing list commits@airflow.incubator.apache.org Received: (qmail 14378 invoked by uid 99); 31 Oct 2018 10:45:17 -0000 Received: from ec2-52-202-80-70.compute-1.amazonaws.com (HELO gitbox.apache.org) (52.202.80.70) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 31 Oct 2018 10:45:17 +0000 From: GitBox To: commits@airflow.apache.org Subject: [GitHub] yangaws commented on a change in pull request #4091: [AIRFLOW-2524] Update SageMaker hook, operator and sensor for training, tuning and transform Message-ID: <154098271687.26867.14940750996556362571.gitbox@gitbox.apache.org> Date: Wed, 31 Oct 2018 10:45:16 -0000 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 8bit yangaws commented on a change in pull request #4091: [AIRFLOW-2524] Update SageMaker hook, operator and sensor for training, tuning and transform URL: https://github.com/apache/incubator-airflow/pull/4091#discussion_r229643455 ########## File path: airflow/contrib/hooks/sagemaker_hook.py ########## @@ -16,299 +16,793 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -import copy +import tarfile +import tempfile import time +import os +import collections +import functools +from datetime import datetime + +import botocore.config from botocore.exceptions import ClientError from airflow.exceptions import AirflowException from airflow.contrib.hooks.aws_hook import AwsHook from airflow.hooks.S3_hook import S3Hook +class LogState(object): + STARTING = 1 + WAIT_IN_PROGRESS = 2 + TAILING = 3 + JOB_COMPLETE = 4 + COMPLETE = 5 + + +# Position is a tuple that includes the last read timestamp and the number of items that were read +# at that time. This is used to figure out which event to start with on the next read. +Position = collections.namedtuple('Position', ['timestamp', 'skip']) + + +def argmin(arr, f): + """Return the index, i, in arr that minimizes f(arr[i])""" + m = None + i = None + for idx, item in enumerate(arr): + if item is not None: + if m is None or f(item) < m: + m = f(item) + i = idx + return i + + +def some(arr): + """Return True iff there is an element, a, of arr such that a is not None""" + return functools.reduce(lambda x, y: x or (y is not None), arr, False) + + +def secondary_training_status_changed(current_job_description, prev_job_description): + """ + Returns true if training job's secondary status message has changed. + + :param current_job_description: Current job description, returned from DescribeTrainingJob call. + :type current_job_description: dict + :param prev_job_description: Previous job description, returned from DescribeTrainingJob call. + :type prev_job_description: dict + + :return: Whether the secondary status message of a training job changed or not. + """ + current_secondary_status_transitions = current_job_description.get('SecondaryStatusTransitions') + if current_secondary_status_transitions is None or len(current_secondary_status_transitions) == 0: + return False + + prev_job_secondary_status_transitions = prev_job_description.get('SecondaryStatusTransitions') \ + if prev_job_description is not None else None + + last_message = prev_job_secondary_status_transitions[-1]['StatusMessage'] \ + if prev_job_secondary_status_transitions is not None \ + and len(prev_job_secondary_status_transitions) > 0 else '' + + message = current_job_description['SecondaryStatusTransitions'][-1]['StatusMessage'] + + return message != last_message + + +def secondary_training_status_message(job_description, prev_description): + """ + Returns a string contains start time and the secondary training job status message. + + :param job_description: Returned response from DescribeTrainingJob call + :type job_description: dict + :param prev_description: Previous job description from DescribeTrainingJob call + :type prev_description: dict + + :return: Job status string to be printed. + """ + + if job_description is None or job_description.get('SecondaryStatusTransitions') is None\ + or len(job_description.get('SecondaryStatusTransitions')) == 0: + return '' + + prev_description_secondary_transitions = prev_description.get('SecondaryStatusTransitions')\ + if prev_description is not None else None + prev_transitions_num = len(prev_description['SecondaryStatusTransitions'])\ + if prev_description_secondary_transitions is not None else 0 + current_transitions = job_description['SecondaryStatusTransitions'] + + transitions_to_print = current_transitions[-1:] if len(current_transitions) == prev_transitions_num else \ + current_transitions[prev_transitions_num - len(current_transitions):] + + status_strs = [] + for transition in transitions_to_print: + message = transition['StatusMessage'] + time_str = datetime.utcfromtimestamp( + time.mktime(job_description['LastModifiedTime'].timetuple())).strftime('%Y-%m-%d %H:%M:%S') + status_strs.append('{} {} - {}'.format(time_str, transition['Status'], message)) + + return '\n'.join(status_strs) + + class SageMakerHook(AwsHook): """ Interact with Amazon SageMaker. - sagemaker_conn_id is required for using - the config stored in db for training/tuning """ - non_terminal_states = {'InProgress', 'Stopping', 'Stopped'} + non_terminal_states = {'InProgress', 'Stopping'} + endpoint_non_terminal_states = {'Creating', 'Updating', 'SystemUpdating', + 'RollingBack', 'Deleting'} failed_states = {'Failed'} def __init__(self, - sagemaker_conn_id=None, - use_db_config=False, - region_name=None, - check_interval=5, - max_ingestion_time=None, *args, **kwargs): super(SageMakerHook, self).__init__(*args, **kwargs) - self.sagemaker_conn_id = sagemaker_conn_id - self.use_db_config = use_db_config - self.region_name = region_name - self.check_interval = check_interval - self.max_ingestion_time = max_ingestion_time - self.conn = self.get_conn() + self.s3_hook = S3Hook(aws_conn_id=self.aws_conn_id) + + def expand_role(self, role): + """ + Expand an IAM role name to an IAM role ARN. If role is already an IAM ARN, + no change is made. + + :param role: IAM role name or ARN + :return: IAM role ARN + """ + if '/' in role: + return role + else: + return self.get_iam_conn().get_role(RoleName=role)['Role']['Arn'] + + def tar_and_s3_upload(self, path, key, bucket): + """ + Tar the local file or directory and upload to s3 - def check_for_url(self, s3url): + :param path: local file or directory + :type path: str + :param key: s3 key + :type key: str + :param bucket: s3 bucket + :type bucket: str + :return: None + """ + with tempfile.TemporaryFile() as temp_file: + if os.path.isdir(path): + files = [os.path.join(path, name) for name in os.listdir(path)] + else: + files = [path] + with tarfile.open(mode='w:gz', fileobj=temp_file) as tar_file: + for f in files: + tar_file.add(f, arcname=os.path.basename(f)) + temp_file.seek(0) + self.s3_hook.load_file_obj(temp_file, key, bucket, True) + + def configure_s3_resources(self, config): + """ + Extract the S3 operations from the configuration and execute them. + + :param config: config of SageMaker operation + :type config: dict + :return: dict """ - check if the s3url exists + s3_operations = config.pop('S3Operations', None) + + if s3_operations is not None: + create_bucket_ops = s3_operations.get('S3CreateBucket') + upload_ops = s3_operations.get('S3Upload') + if create_bucket_ops: + for op in create_bucket_ops: + self.s3_hook.create_bucket(bucket_name=op['Bucket']) + if upload_ops: + for op in upload_ops: + if op['Tar']: + self.tar_and_s3_upload(op['Path'], op['Key'], + op['Bucket']) + else: + self.s3_hook.load_file(op['Path'], op['Key'], + op['Bucket']) + + return config + + def check_s3_url(self, s3url): + """ + Check if an S3 URL exists + :param s3url: S3 url :type s3url:str :return: bool """ bucket, key = S3Hook.parse_s3_url(s3url) - s3hook = S3Hook(aws_conn_id=self.aws_conn_id) - if not s3hook.check_for_bucket(bucket_name=bucket): + if not self.s3_hook.check_for_bucket(bucket_name=bucket): raise AirflowException( "The input S3 Bucket {} does not exist ".format(bucket)) - if key and not s3hook.check_for_key(key=key, bucket_name=bucket)\ - and not s3hook.check_for_prefix( + if key and not self.s3_hook.check_for_key(key=key, bucket_name=bucket)\ + and not self.s3_hook.check_for_prefix( prefix=key, bucket_name=bucket, delimiter='/'): # check if s3 key exists in the case user provides a single file - # or if s3 prefix exists in the case user provides a prefix for files + # or if s3 prefix exists in the case user provides multiple files in + # a prefix raise AirflowException("The input S3 Key " "or Prefix {} does not exist in the Bucket {}" .format(s3url, bucket)) return True - def check_valid_training_input(self, training_config): + def check_training_config(self, training_config): """ - Run checks before a training starts + Check if a training configuration is valid + :param training_config: training_config :type training_config: dict :return: None """ for channel in training_config['InputDataConfig']: - self.check_for_url(channel['DataSource'] - ['S3DataSource']['S3Uri']) + self.check_s3_url(channel['DataSource'] + ['S3DataSource']['S3Uri']) - def check_valid_tuning_input(self, tuning_config): + def check_tuning_config(self, tuning_config): """ - Run checks before a tuning job starts + Check if a tuning configuration is valid + :param tuning_config: tuning_config :type tuning_config: dict :return: None """ for channel in tuning_config['TrainingJobDefinition']['InputDataConfig']: - self.check_for_url(channel['DataSource'] - ['S3DataSource']['S3Uri']) + self.check_s3_url(channel['DataSource'] + ['S3DataSource']['S3Uri']) - def check_status(self, non_terminal_states, - failed_state, key, - describe_function, *args): - """ - :param non_terminal_states: the set of non_terminal states - :type non_terminal_states: set - :param failed_state: the set of failed states - :type failed_state: set - :param key: the key of the response dict - that points to the state - :type key: str - :param describe_function: the function used to retrieve the status - :type describe_function: python callable - :param args: the arguments for the function - :return: None + def get_conn(self): """ - sec = 0 - running = True - - while running: + Establish an AWS connection for SageMaker - sec = sec + self.check_interval - - if self.max_ingestion_time and sec > self.max_ingestion_time: - # ensure that the job gets killed if the max ingestion time is exceeded - raise AirflowException("SageMaker job took more than " - "%s seconds", self.max_ingestion_time) - - time.sleep(self.check_interval) - try: - response = describe_function(*args) - status = response[key] - self.log.info("Job still running for %s seconds... " - "current status is %s" % (sec, status)) - except KeyError: - raise AirflowException("Could not get status of the SageMaker job") - except ClientError: - raise AirflowException("AWS request failed, check log for more info") + :return: a boto3 SageMaker client + """ + return self.get_client_type('sagemaker') - if status in non_terminal_states: - running = True - elif status in failed_state: - raise AirflowException("SageMaker job failed because %s" - % response['FailureReason']) - else: - running = False + def get_log_conn(self): + """ + Establish an AWS connection for retrieving logs during training - self.log.info('SageMaker Job Compeleted') + :return: a boto3 CloudWatchLog client + """ + config = botocore.config.Config(retries={'max_attempts': 15}) + return self.get_client_type('logs', config=config) - def get_conn(self): + def get_iam_conn(self): """ - Establish an AWS connection - :return: a boto3 SageMaker client + Establish an AWS connection for retrieving IAM roles during training + + :return: a boto3 IAM client """ - return self.get_client_type('sagemaker', region_name=self.region_name) + return self.get_client_type('iam') - def list_training_job(self, name_contains=None, status_equals=None): + def log_stream(self, log_group, stream_name, start_time=0, skip=0): """ - List the training jobs associated with the given input - :param name_contains: A string in the training job name - :type name_contains: str - :param status_equals: 'InProgress'|'Completed' - |'Failed'|'Stopping'|'Stopped' - :return:dict + A generator for log items in a single stream. This will yield all the + items that are available at the current moment. + + :param log_group: The name of the log group. + :type log_group: str + :param stream_name: The name of the specific stream. + :type stream_name: str + :param start_time: The time stamp value to start reading the logs from (default: 0). + :type start_time: int + :param skip: The number of log entries to skip at the start (default: 0). + This is for when there are multiple entries at the same timestamp. + :type skip: int + :return:A CloudWatch log event with the following key-value pairs: + 'timestamp' (int): The time of the event. + 'message' (str): The log event data. + 'ingestionTime' (int): The time the event was ingested. """ - return self.conn.list_training_jobs( - NameContains=name_contains, StatusEquals=status_equals) - def list_tuning_job(self, name_contains=None, status_equals=None): + next_token = None + + event_count = 1 + while event_count > 0: + if next_token is not None: + token_arg = {'nextToken': next_token} + else: + token_arg = {} + + response = self.get_log_conn().get_log_events(logGroupName=log_group, + logStreamName=stream_name, + startTime=start_time, + startFromHead=True, + **token_arg) + next_token = response['nextForwardToken'] + events = response['events'] + event_count = len(events) + if event_count > skip: + events = events[skip:] + skip = 0 + else: + skip = skip - event_count + events = [] + for ev in events: + yield ev + + def multi_stream_iter(self, log_group, streams, positions=None): """ - List the tuning jobs associated with the given input - :param name_contains: A string in the training job name - :type name_contains: str - :param status_equals: 'InProgress'|'Completed' - |'Failed'|'Stopping'|'Stopped' - :return:dict + Iterate over the available events coming from a set of log streams in a single log group + interleaving the events from each stream so they're yielded in timestamp order. + + :param log_group: The name of the log group. + :type log_group: str + :param streams: A list of the log stream names. The position of the stream in this list is + the stream number. + :type streams: list + :param positions: A list of pairs of (timestamp, skip) which represents the last record + read from each stream. + :type positions: list + :return: A tuple of (stream number, cloudwatch log event). """ - return self.conn.list_hyper_parameter_tuning_job( - NameContains=name_contains, StatusEquals=status_equals) + positions = positions or {s: Position(timestamp=0, skip=0) for s in streams} + event_iters = [self.log_stream(log_group, s, positions[s].timestamp, positions[s].skip) + for s in streams] + events = [next(s) if s else None for s in event_iters] + + while some(events): + i = argmin(events, lambda x: x['timestamp'] if x else 9999999999) + yield (i, events[i]) + try: + events[i] = next(event_iters[i]) + except StopIteration: + events[i] = None - def create_training_job(self, training_job_config, wait_for_completion=True): + def create_training_job(self, config, wait_for_completion=True, print_log=True, + check_interval=30, max_ingestion_time=None): """ Create a training job - :param training_job_config: the config for training - :type training_job_config: dict + + :param config: the config for training + :type config: dict :param wait_for_completion: if the program should keep running until job finishes :type wait_for_completion: bool - :return: A dict that contains ARN of the training job. + :param check_interval: the time interval in seconds which the operator + will check the status of any SageMaker job + :type check_interval: int + :param max_ingestion_time: the maximum ingestion time in seconds. Any + SageMaker jobs that run longer than this will fail. Setting this to + None implies no timeout for any SageMaker job. + :type max_ingestion_time: int + :return: A response to training job creation """ - if self.use_db_config: - if not self.sagemaker_conn_id: - raise AirflowException("SageMaker connection id must be present to read \ - SageMaker training jobs configuration.") - sagemaker_conn = self.get_connection(self.sagemaker_conn_id) - - config = copy.deepcopy(sagemaker_conn.extra_dejson) - training_job_config.update(config) - self.check_valid_training_input(training_job_config) + self.check_training_config(config) + + response = self.get_conn().create_training_job(**config) + if print_log: + self.check_training_status_with_log(config['TrainingJobName'], + SageMakerHook.non_terminal_states, + SageMakerHook.failed_states, + wait_for_completion, + check_interval, max_ingestion_time + ) + elif wait_for_completion: + describe_response = self.check_status(config['TrainingJobName'], + SageMakerHook.non_terminal_states, + SageMakerHook.failed_states, + 'TrainingJobStatus', + self.describe_training_job, + check_interval, max_ingestion_time + ) + + billable_time = \ + (describe_response['TrainingEndTime'] - describe_response['TrainingStartTime']) * \ + describe_response['ResourceConfig']['InstanceCount'] + self.log.info('Billable seconds:{}'.format(int(billable_time.total_seconds()) + 1)) - response = self.conn.create_training_job( - **training_job_config) - if wait_for_completion: - self.check_status(SageMakerHook.non_terminal_states, - SageMakerHook.failed_states, - 'TrainingJobStatus', - self.describe_training_job, - training_job_config['TrainingJobName']) return response - def create_tuning_job(self, tuning_job_config, wait_for_completion=True): + def create_tuning_job(self, config, wait_for_completion=True, + check_interval=30, max_ingestion_time=None): """ Create a tuning job - :param tuning_job_config: the config for tuning - :type tuning_job_config: dict + + :param config: the config for tuning + :type config: dict :param wait_for_completion: if the program should keep running until job finishes :param wait_for_completion: bool - :return: A dict that contains ARN of the tuning job. + :param check_interval: the time interval in seconds which the operator + will check the status of any SageMaker job + :type check_interval: int + :param max_ingestion_time: the maximum ingestion time in seconds. Any + SageMaker jobs that run longer than this will fail. Setting this to + None implies no timeout for any SageMaker job. + :type max_ingestion_time: int + :return: A response to tuning job creation """ - if self.use_db_config: - if not self.sagemaker_conn_id: - raise AirflowException( - "SageMaker connection id must be present to \ - read SageMaker tunning job configuration.") - sagemaker_conn = self.get_connection(self.sagemaker_conn_id) + self.check_tuning_config(config) - config = sagemaker_conn.extra_dejson.copy() - tuning_job_config.update(config) - - self.check_valid_tuning_input(tuning_job_config) - - response = self.conn.create_hyper_parameter_tuning_job( - **tuning_job_config) + response = self.get_conn().create_hyper_parameter_tuning_job(**config) if wait_for_completion: - self.check_status(SageMakerHook.non_terminal_states, + self.check_status(config['HyperParameterTuningJobName'], + SageMakerHook.non_terminal_states, SageMakerHook.failed_states, 'HyperParameterTuningJobStatus', self.describe_tuning_job, - tuning_job_config['HyperParameterTuningJobName']) + check_interval, max_ingestion_time + ) return response - def create_transform_job(self, transform_job_config, wait_for_completion=True): + def create_transform_job(self, config, wait_for_completion=True, + check_interval=30, max_ingestion_time=None): """ Create a transform job - :param transform_job_config: the config for transform job - :type transform_job_config: dict - :param wait_for_completion: - if the program should keep running until job finishes + + :param config: the config for transform job + :type config: dict + :param wait_for_completion: if the program should keep running until job finishes :type wait_for_completion: bool - :return: A dict that contains ARN of the transform job. + :param check_interval: the time interval in seconds which the operator + will check the status of any SageMaker job + :type check_interval: int + :param max_ingestion_time: the maximum ingestion time in seconds. Any + SageMaker jobs that run longer than this will fail. Setting this to + None implies no timeout for any SageMaker job. + :type max_ingestion_time: int + :return: A response to transform job creation """ - if self.use_db_config: - if not self.sagemaker_conn_id: - raise AirflowException( - "SageMaker connection id must be present to \ - read SageMaker transform job configuration.") - - sagemaker_conn = self.get_connection(self.sagemaker_conn_id) - config = sagemaker_conn.extra_dejson.copy() - transform_job_config.update(config) + self.check_s3_url(config + ['TransformInput']['DataSource'] + ['S3DataSource']['S3Uri']) - self.check_for_url(transform_job_config - ['TransformInput']['DataSource'] - ['S3DataSource']['S3Uri']) - - response = self.conn.create_transform_job( - **transform_job_config) + response = self.get_conn().create_transform_job(**config) if wait_for_completion: - self.check_status(SageMakerHook.non_terminal_states, + self.check_status(config['TransformJobName'], + SageMakerHook.non_terminal_states, SageMakerHook.failed_states, 'TransformJobStatus', self.describe_transform_job, - transform_job_config['TransformJobName']) + check_interval, max_ingestion_time + ) return response - def create_model(self, model_config): + def create_model(self, config): """ Create a model job - :param model_config: the config for model - :type model_config: dict - :return: A dict that contains ARN of the model. + + :param config: the config for model + :type config: dict + :return: A response to model creation + """ + + return self.get_conn().create_model(**config) + + def create_endpoint_config(self, config): + """ + Create an endpoint config + + :param config: the config for endpoint-config + :type config: dict + :return: A response to endpoint config creation + """ + + return self.get_conn().create_endpoint_config(**config) + + def create_endpoint(self, config, wait_for_completion=True, + check_interval=30, max_ingestion_time=None): + """ + Create an endpoint + + :param config: the config for endpoint + :type config: dict + :param wait_for_completion: if the program should keep running until job finishes + :type wait_for_completion: bool + :param check_interval: the time interval in seconds which the operator + will check the status of any SageMaker job + :type check_interval: int + :param max_ingestion_time: the maximum ingestion time in seconds. Any + SageMaker jobs that run longer than this will fail. Setting this to + None implies no timeout for any SageMaker job. + :type max_ingestion_time: int + :return: A response to endpoint creation + """ + + response = self.get_conn().create_endpoint(**config) + if wait_for_completion: + self.check_status(config['EndpointName'], + SageMakerHook.endpoint_non_terminal_states, + SageMakerHook.failed_states, + 'EndpointStatus', + self.describe_endpoint, + check_interval, max_ingestion_time + ) + return response + + def update_endpoint(self, config, wait_for_completion=True, + check_interval=30, max_ingestion_time=None): + """ + Update an endpoint + + :param config: the config for endpoint + :type config: dict + :param wait_for_completion: if the program should keep running until job finishes + :type wait_for_completion: bool + :param check_interval: the time interval in seconds which the operator + will check the status of any SageMaker job + :type check_interval: int + :param max_ingestion_time: the maximum ingestion time in seconds. Any + SageMaker jobs that run longer than this will fail. Setting this to + None implies no timeout for any SageMaker job. + :type max_ingestion_time: int + :return: A response to endpoint update """ - return self.conn.create_model( - **model_config) + response = self.get_conn().update_endpoint(**config) + if wait_for_completion: + self.check_status(config['EndpointName'], + SageMakerHook.non_terminal_states, + SageMakerHook.failed_states, + 'EndpointStatus', + self.describe_endpoint, + check_interval, max_ingestion_time + ) + return response - def describe_training_job(self, training_job_name): + def describe_training_job(self, name): """ - :param training_job_name: the name of the training job - :type training_job_name: str - Return the training job info associated with the current job_name + Return the training job info associated with the name + + :param name: the name of the training job + :type name: str :return: A dict contains all the training job info """ - return self.conn\ - .describe_training_job(TrainingJobName=training_job_name) - def describe_tuning_job(self, tuning_job_name): + return self.get_conn().describe_training_job(TrainingJobName=name) + + def describe_training_job_with_log(self, job_name, non_terminal_states, positions, stream_names, + instance_count, state, last_description, + last_describe_job_call): + """ + Return the training job info associated with job_name and print CloudWatch logs + """ + log_group = '/aws/sagemaker/TrainingJobs' + + if len(stream_names) < instance_count: + # Log streams are created whenever a container starts writing to stdout/err, so this list + # may be dynamic until we have a stream for every instance. + try: + streams = self.get_log_conn().describe_log_streams( + logGroupName=log_group, + logStreamNamePrefix=job_name + '/', + orderBy='LogStreamName', + limit=instance_count + ) + stream_names = [s['logStreamName'] for s in streams['logStreams']] + positions.update([(s, Position(timestamp=0, skip=0)) + for s in stream_names if s not in positions]) + except ClientError as e: + # On the very first training job run on an account, there's no log group until + # the container starts logging, so ignore any errors thrown about that + err = e.response.get('Error', {}) + if err.get('Code', None) != 'ResourceNotFoundException': + raise + + if len(stream_names) > 0: + for idx, event in self.multi_stream_iter(log_group, stream_names, positions): + self.log.info(event['message']) + ts, count = positions[stream_names[idx]] + if event['timestamp'] == ts: + positions[stream_names[idx]] = Position(timestamp=ts, skip=count + 1) + else: + positions[stream_names[idx]] = Position(timestamp=event['timestamp'], skip=1) + + if state == LogState.COMPLETE: + return state, last_description, last_describe_job_call + + if state == LogState.JOB_COMPLETE: + state = LogState.COMPLETE + elif time.time() - last_describe_job_call >= 30: + description = self.describe_training_job(job_name) + last_describe_job_call = time.time() + + if secondary_training_status_changed(description, last_description): + self.log.info(secondary_training_status_message(description, last_description)) + last_description = description + + status = description['TrainingJobStatus'] + + if status not in non_terminal_states: + state = LogState.JOB_COMPLETE + return state, last_description, last_describe_job_call + + def describe_tuning_job(self, name): """ - :param tuning_job_name: the name of the tuning job - :type tuning_job_name: string - Return the tuning job info associated with the current job_name + Return the tuning job info associated with the name + + :param name: the name of the tuning job + :type name: string :return: A dict contains all the tuning job info """ - return self.conn\ - .describe_hyper_parameter_tuning_job( - HyperParameterTuningJobName=tuning_job_name) - def describe_transform_job(self, transform_job_name): + return self.get_conn().describe_hyper_parameter_tuning_job( + HyperParameterTuningJobName=name) + + def describe_model(self, name): + """ + Return the SageMaker model info associated with the name + + :param name: the name of the SageMaker model + :type name: string + :return: A dict contains all the model info + """ + + return self.get_conn().describe_model(ModelName=name) + + def describe_transform_job(self, name): """ - :param transform_job_name: the name of the transform job - :type transform_job_name: string - Return the transform job info associated with the current job_name + Return the transform job info associated with the name + + :param name: the name of the transform job + :type name: string :return: A dict contains all the transform job info """ - return self.conn\ - .describe_transform_job( - TransformJobName=transform_job_name) + + return self.get_conn().describe_transform_job(TransformJobName=name) + + def describe_endpoint_config(self, name): + """ + Return the endpoint config info associated with the name + + :param name: the name of the endpoint config + :type name: string + :return: A dict contains all the endpoint config info + """ + + return self.get_conn().describe_endpoint_config(EndpointConfigName=name) + + def describe_endpoint(self, name): + """ + :param name: the name of the endpoint + :type name: string + :return: A dict contains all the endpoint info + """ + + return self.get_conn().describe_endpoint(EndpointName=name) + + def check_status(self, job_name, non_terminal_states, + failed_states, key, + describe_function, check_interval, + max_ingestion_time): + """ + Check status of a SageMaker job + + :param job_name: name of the job to check status + :type job_name: str + :param non_terminal_states: the set of non_terminal states + :type non_terminal_states: set + :param failed_states: the set of failed states + :type failed_states: set + :param key: the key of the response dict + that points to the state + :type key: str + :param describe_function: the function used to retrieve the status + :type describe_function: python callable + :param args: the arguments for the function + :param check_interval: the time interval in seconds which the operator + will check the status of any SageMaker job + :type check_interval: int + :param max_ingestion_time: the maximum ingestion time in seconds. Any + SageMaker jobs that run longer than this will fail. Setting this to + None implies no timeout for any SageMaker job. + :type max_ingestion_time: int + :return: response of describe call after job is done + """ + sec = 0 + running = True + + while running: + time.sleep(check_interval) + sec = sec + check_interval + + if max_ingestion_time and sec > max_ingestion_time: + # ensure that the job gets killed if the max ingestion time is exceeded + raise AirflowException('SageMaker job took more than ' + '%s seconds', max_ingestion_time) + try: + response = describe_function(job_name) + status = response[key] + self.log.info('Job still running for %s seconds... ' + 'current status is %s' % (sec, status)) + except KeyError: + raise AirflowException('Could not get status of the SageMaker job') + except ClientError: + raise AirflowException('AWS request failed, check logs for more info') + + if status in non_terminal_states: + running = True + elif status in failed_states: + raise AirflowException('SageMaker job failed because %s' + % response['FailureReason']) Review comment: Updated. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: users@infra.apache.org With regards, Apache Git Services