Skip to content

Commit

Permalink
[AIRFLOW-5027] Grab CloudWatch logs after ECS task has finished (apac…
Browse files Browse the repository at this point in the history
  • Loading branch information
StevenReitsma authored and Jing Guo committed Sep 2, 2019
1 parent f114cb4 commit d237f45
Show file tree
Hide file tree
Showing 6 changed files with 251 additions and 56 deletions.
102 changes: 102 additions & 0 deletions airflow/contrib/hooks/aws_logs_hook.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,102 @@
# -*- coding: utf-8 -*-
#
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.

"""
This module contains a hook (AwsLogsHook) with some very basic
functionality for interacting with AWS CloudWatch.
"""

from airflow.contrib.hooks.aws_hook import AwsHook


class AwsLogsHook(AwsHook):
"""
Interact with AWS CloudWatch Logs
:param region_name: AWS Region Name (example: us-west-2)
:type region_name: str
"""

def __init__(self, region_name=None, *args, **kwargs):
self.region_name = region_name
super().__init__(*args, **kwargs)

def get_conn(self):
"""
Establish an AWS connection for retrieving logs.
:rtype: CloudWatchLogs.Client
"""
return self.get_client_type('logs', region_name=self.region_name)

def get_log_events(self, log_group, log_stream_name, start_time=0, skip=0, start_from_head=True):
"""
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 log_stream_name: The name of the specific stream.
:type log_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
:param start_from_head: whether to start from the beginning (True) of the log or
at the end of the log (False).
:type start_from_head: bool
:rtype: dict
:return: | A CloudWatch log event with the following key-value pairs:
| 'timestamp' (int): The time in milliseconds of the event.
| 'message' (str): The log event data.
| 'ingestionTime' (int): The time in milliseconds the event was ingested.
"""

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_conn().get_log_events(logGroupName=log_group,
logStreamName=log_stream_name,
startTime=start_time,
startFromHead=start_from_head,
**token_arg)

events = response['events']
event_count = len(events)

if event_count > skip:
events = events[skip:]
skip = 0
else:
skip = skip - event_count
events = []

yield from events

if 'nextForwardToken' in response:
next_token = response['nextForwardToken']
else:
return
69 changes: 20 additions & 49 deletions airflow/contrib/hooks/sagemaker_hook.py
Original file line number Diff line number Diff line change
Expand Up @@ -21,12 +21,13 @@
import time
import os
import collections
import warnings

import botocore.config
from botocore.exceptions import ClientError

from airflow.exceptions import AirflowException
from airflow.contrib.hooks.aws_hook import AwsHook
from airflow.contrib.hooks.aws_logs_hook import AwsLogsHook
from airflow.hooks.S3_hook import S3Hook
from airflow.utils import timezone

Expand Down Expand Up @@ -130,6 +131,7 @@ def __init__(self,
*args, **kwargs):
super().__init__(*args, **kwargs)
self.s3_hook = S3Hook(aws_conn_id=self.aws_conn_id)
self.logs_hook = AwsLogsHook(aws_conn_id=self.aws_conn_id)

def tar_and_s3_upload(self, path, key, bucket):
"""
Expand Down Expand Up @@ -232,58 +234,27 @@ def get_conn(self):

def get_log_conn(self):
"""
Establish an AWS connection for retrieving logs during training
:rtype: CloudWatchLogs.Client
This method is deprecated.
Please use :py:meth:`airflow.contrib.hooks.AwsLogsHook.get_conn` instead.
"""
config = botocore.config.Config(retries={'max_attempts': 15})
return self.get_client_type('logs', config=config)
warnings.warn("Method `get_log_conn` has been deprecated. "
"Please use `airflow.contrib.hooks.AwsLogsHook.get_conn` instead.",
category=DeprecationWarning,
stacklevel=2)

return self.logs_hook.get_conn()

def log_stream(self, log_group, stream_name, start_time=0, skip=0):
"""
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
:rtype: dict
:return: | A CloudWatch log event with the following key-value pairs:
| 'timestamp' (int): The time in milliseconds of the event.
| 'message' (str): The log event data.
| 'ingestionTime' (int): The time in milliseconds the event was ingested.
This method is deprecated.
Please use :py:meth:`airflow.contrib.hooks.AwsLogsHook.get_log_events` instead.
"""
warnings.warn("Method `log_stream` has been deprecated. "
"Please use `airflow.contrib.hooks.AwsLogsHook.get_log_events` instead.",
category=DeprecationWarning,
stacklevel=2)

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 = []
yield from events
return self.logs_hook.get_log_events(log_group, stream_name, start_time, skip)

def multi_stream_iter(self, log_group, streams, positions=None):
"""
Expand All @@ -301,7 +272,7 @@ def multi_stream_iter(self, log_group, streams, positions=None):
:return: A tuple of (stream number, cloudwatch log event).
"""
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)
event_iters = [self.logs_hook.get_log_events(log_group, s, positions[s].timestamp, positions[s].skip)
for s in streams]
events = []
for s in event_iters:
Expand Down Expand Up @@ -526,7 +497,7 @@ def describe_training_job_with_log(self, job_name, positions, stream_names,
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.
logs_conn = self.get_log_conn()
logs_conn = self.logs_hook.get_conn()
try:
streams = logs_conn.describe_log_streams(
logGroupName=log_group,
Expand Down
40 changes: 39 additions & 1 deletion airflow/contrib/operators/ecs_operator.py
Original file line number Diff line number Diff line change
Expand Up @@ -20,12 +20,14 @@
from typing_extensions import Protocol
import sys
import re
from datetime import datetime

from airflow.exceptions import AirflowException
from airflow.models import BaseOperator
from airflow.utils.decorators import apply_defaults

from airflow.contrib.hooks.aws_hook import AwsHook
from airflow.contrib.hooks.aws_logs_hook import AwsLogsHook


class ECSProtocol(Protocol):
Expand Down Expand Up @@ -71,6 +73,19 @@ class ECSOperator(BaseOperator):
:type platform_version: str
:param network_configuration: the network configuration for the task
:type network_configuration: dict
:param awslogs_group: the CloudWatch group where your ECS container logs are stored.
Only required if you want logs to be shown in the Airflow UI after your job has
finished.
:type awslogs_group: str
:param awslogs_region: the region in which your CloudWatch logs are stored.
If None, this is the same as the `region_name` parameter. If that is also None,
this is the default AWS region based on your connection settings.
:type awslogs_region: str
:param awslogs_stream_prefix: the stream prefix that is used for the CloudWatch logs.
This is usually based on some custom name combined with the name of the container.
Only required if you want logs to be shown in the Airflow UI after your job has
finished.
:type awslogs_stream_prefix: str
"""

ui_color = '#f0ede4'
Expand All @@ -82,7 +97,8 @@ class ECSOperator(BaseOperator):
def __init__(self, task_definition, cluster, overrides,
aws_conn_id=None, region_name=None, launch_type='EC2',
group=None, placement_constraints=None, platform_version='LATEST',
network_configuration=None, **kwargs):
network_configuration=None, awslogs_group=None,
awslogs_region=None, awslogs_stream_prefix=None, **kwargs):
super().__init__(**kwargs)

self.aws_conn_id = aws_conn_id
Expand All @@ -96,6 +112,13 @@ def __init__(self, task_definition, cluster, overrides,
self.platform_version = platform_version
self.network_configuration = network_configuration

self.awslogs_group = awslogs_group
self.awslogs_stream_prefix = awslogs_stream_prefix
self.awslogs_region = awslogs_region

if self.awslogs_region is None:
self.awslogs_region = region_name

self.hook = self.get_hook()

def execute(self, context):
Expand Down Expand Up @@ -154,6 +177,15 @@ def _check_success_task(self):
)
self.log.info('ECS Task stopped, check status: %s', response)

# Get logs from CloudWatch if the awslogs log driver was used
if self.awslogs_group and self.awslogs_stream_prefix:
self.log.info('ECS Task logs output:')
task_id = self.arn.split("/")[-1]
stream_name = "{}/{}".format(self.awslogs_stream_prefix, task_id)
for event in self.get_logs_hook().get_log_events(self.awslogs_group, stream_name):
dt = datetime.fromtimestamp(event['timestamp'] / 1000.0)
self.log.info("[{}] {}".format(dt.isoformat(), event['message']))

if len(response.get('failures', [])) > 0:
raise AirflowException(response)

Expand Down Expand Up @@ -185,6 +217,12 @@ def get_hook(self):
aws_conn_id=self.aws_conn_id
)

def get_logs_hook(self):
return AwsLogsHook(
aws_conn_id=self.aws_conn_id,
region_name=self.awslogs_region
)

def on_kill(self):
response = self.client.stop_task(
cluster=self.cluster,
Expand Down
82 changes: 82 additions & 0 deletions tests/contrib/hooks/test_aws_logs_hook.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,82 @@
# -*- coding: utf-8 -*-
#
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
#

import unittest

from airflow.contrib.hooks.aws_logs_hook import AwsLogsHook

try:
from moto import mock_logs
except ImportError:
mock_logs = None


class TestAwsLogsHook(unittest.TestCase):

@unittest.skipIf(mock_logs is None, 'mock_logs package not present')
@mock_logs
def test_get_conn_returns_a_boto3_connection(self):
hook = AwsLogsHook(aws_conn_id='aws_default',
region_name="us-east-1")
self.assertIsNotNone(hook.get_conn())

@unittest.skipIf(mock_logs is None, 'mock_logs package not present')
# moto.logs does not support proper pagination so we cannot test that yet
# https://github.com/spulec/moto/issues/2259
@mock_logs
def test_get_log_events(self):
log_group_name = 'example-group'
log_stream_name = 'example-log-stream'

hook = AwsLogsHook(aws_conn_id='aws_default',
region_name="us-east-1")

# First we create some log events
conn = hook.get_conn()
conn.create_log_group(logGroupName=log_group_name)
conn.create_log_stream(logGroupName=log_group_name, logStreamName=log_stream_name)

input_events = [
{
'timestamp': 1,
'message': 'Test Message 1'
}
]

conn.put_log_events(logGroupName=log_group_name,
logStreamName=log_stream_name,
logEvents=input_events)

events = hook.get_log_events(
log_group=log_group_name,
log_stream_name=log_stream_name
)

# Iterate through entire generator
events = list(events)
count = len(events)

assert count == 1
assert events[0]['timestamp'] == input_events[0]['timestamp']
assert events[0]['message'] == input_events[0]['message']


if __name__ == '__main__':
unittest.main()
Loading

0 comments on commit d237f45

Please sign in to comment.