From commits-return-26342-archive-asf-public=cust-asf.ponee.io@airflow.incubator.apache.org Sat Oct 27 23:58:54 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 5FC6D1807D4 for ; Sat, 27 Oct 2018 23:58:49 +0200 (CEST) Received: (qmail 44058 invoked by uid 500); 27 Oct 2018 21:58:42 -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 43532 invoked by uid 99); 27 Oct 2018 21:58:42 -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; Sat, 27 Oct 2018 21:58:42 +0000 From: GitBox To: commits@airflow.apache.org Subject: [GitHub] ashb commented on a change in pull request #4091: [AIRFLOW-2524] Update SageMaker hook, operator and sensor for training, tuning and transform Message-ID: <154067752191.2159.390873367241997180.gitbox@gitbox.apache.org> Date: Sat, 27 Oct 2018 21:58:41 -0000 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 8bit ashb 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_r228726249 ########## 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) Review comment: I'm somewhat surprised there isn't a CloudWatchLogs.Paginator for get_log_events in boto3. That said, given we're passing `logStreamNames=stream_name` couldn't we use something like: ``` paginator = self.get_log_conn().get_paginator('filter_log_events') response = paginator.paginate(logGroupName=log_group, logStreamName=stream_name, startTime=start_time, startFromHead=True) for page in response: events = page['events'] ... ``` (i.e. remove the next_token/token_arg etc by using boto3 Paginators). I'm not sure the `while event_count > 0`: would be needed then either. I'm not familiar enough with the CloudWatch Logs APIs to say if get_log_events and filter_log_events do the same here. ---------------------------------------------------------------- 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