shipyard/shipyard_airflow/plugins/drydock_base_operator.py
Anthony Lin b9b0e27de0 Add UCP Base Operator
1) Refactor Drydock Base Operator to make use of the
   UCP Base Operator instead

2) Dump logs from Drydock Pods when there are Exceptions

Change-Id: I3fbe03d13b5fc89a503cfb2c3c25751076718554
2018-04-18 14:19:16 +00:00

353 lines
13 KiB
Python

# Copyright 2018 AT&T Intellectual Property. All other rights reserved.
#
# Licensed 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 json
import logging
import os
import time
from urllib.parse import urlparse
from airflow.exceptions import AirflowException
from airflow.plugins_manager import AirflowPlugin
from airflow.utils.decorators import apply_defaults
import drydock_provisioner.drydock_client.client as client
import drydock_provisioner.drydock_client.session as session
from drydock_provisioner import error as errors
from service_endpoint import ucp_service_endpoint
from service_token import shipyard_service_token
from ucp_base_operator import UcpBaseOperator
LOG = logging.getLogger(__name__)
class DrydockBaseOperator(UcpBaseOperator):
"""Drydock Base Operator
All drydock related workflow operators will use the drydock
base operator as the parent and inherit attributes and methods
from this class
"""
@apply_defaults
def __init__(self,
deckhand_design_ref=None,
deckhand_svc_type='deckhand',
drydock_client=None,
drydock_svc_endpoint=None,
drydock_svc_type='physicalprovisioner',
drydock_task_id=None,
node_filter=None,
redeploy_server=None,
svc_session=None,
svc_token=None,
*args, **kwargs):
"""Initialization of DrydockBaseOperator object.
:param deckhand_design_ref: A URI reference to the design documents
:param deckhand_svc_type: Deckhand Service Type
:param drydockclient: An instance of drydock client
:param drydock_svc_endpoint: Drydock Service Endpoint
:param drydock_svc_type: Drydock Service Type
:param drydock_task_id: Drydock Task ID
:param node_filter: A filter for narrowing the scope of the task.
Valid fields are 'node_names', 'rack_names',
'node_tags'. Note that node filter is turned
off by default, i.e. all nodes will be deployed.
:param redeploy_server: Server to be redeployed
:param svc_session: Keystone Session
:param svc_token: Keystone Token
The Drydock operator assumes that prior steps have set xcoms for
the action and the deployment configuration
"""
super(DrydockBaseOperator,
self).__init__(
pod_selector_pattern=[{'pod_pattern': 'drydock-api',
'container': 'drydock-api'}],
*args, **kwargs)
self.deckhand_design_ref = deckhand_design_ref
self.deckhand_svc_type = deckhand_svc_type
self.drydock_client = drydock_client
self.drydock_svc_endpoint = drydock_svc_endpoint
self.drydock_svc_type = drydock_svc_type
self.drydock_task_id = drydock_task_id
self.node_filter = node_filter
self.redeploy_server = redeploy_server
self.svc_session = svc_session
self.svc_token = svc_token
def run_base(self, context):
# Logs uuid of action performed by the Operator
LOG.info("DryDock Operator for action %s", self.action_info['id'])
# Retrieve information of the server that we want to redeploy if user
# executes the 'redeploy_server' dag
# Set node filter to be the server that we want to redeploy
if self.action_info['dag_id'] == 'redeploy_server':
self.redeploy_server = (
self.action_info['parameters']['server-name'])
if self.redeploy_server:
LOG.info("Server to be redeployed is %s",
self.redeploy_server)
self.node_filter = self.redeploy_server
else:
raise AirflowException('%s was unable to retrieve the '
'server to be redeployed.'
% self.__class__.__name__)
# Retrieve Endpoint Information
self.drydock_svc_endpoint = ucp_service_endpoint(
self, svc_type=self.drydock_svc_type)
LOG.info("Drydock endpoint is %s", self.drydock_svc_endpoint)
# Parse DryDock Service Endpoint
drydock_url = urlparse(self.drydock_svc_endpoint)
# Build a DrydockSession with credentials and target host
# information.
# The DrydockSession will care for TCP connection pooling
# and header management
LOG.info("Build DryDock Session")
dd_session = session.DrydockSession(drydock_url.hostname,
port=drydock_url.port,
auth_gen=self._auth_gen)
# Raise Exception if we are not able to set up the session
if dd_session:
LOG.info("Successfully Set Up DryDock Session")
else:
raise AirflowException("Failed to set up Drydock Session!")
# Use the DrydockSession to build a DrydockClient that can
# be used to make one or more API calls
LOG.info("Create DryDock Client")
self.drydock_client = client.DrydockClient(dd_session)
# Raise Exception if we are not able to build the client
if self.drydock_client:
LOG.info("Successfully Set Up DryDock client")
else:
raise AirflowException("Failed to set up Drydock Client!")
# Retrieve DeckHand Endpoint Information
deckhand_svc_endpoint = ucp_service_endpoint(
self, svc_type=self.deckhand_svc_type)
LOG.info("Deckhand endpoint is %s", deckhand_svc_endpoint)
# Retrieve last committed revision id
committed_revision_id = self.xcom_puller.get_design_version()
# Form DeckHand Design Reference Path
# This URL will be used to retrieve the Site Design YAMLs
deckhand_path = "deckhand+" + deckhand_svc_endpoint
self.deckhand_design_ref = os.path.join(deckhand_path,
"revisions",
str(committed_revision_id),
"rendered-documents")
if self.deckhand_design_ref:
LOG.info("Design YAMLs will be retrieved from %s",
self.deckhand_design_ref)
else:
raise AirflowException("Unable to Retrieve Design Reference!")
@shipyard_service_token
def _auth_gen(self):
# Generator method for the Drydock Session to use to get the
# auth headers necessary
return [('X-Auth-Token', self.svc_token)]
def create_task(self, task_action):
# Initialize Variables
create_task_response = {}
# Node Filter
LOG.info("Nodes Filter List: %s", self.node_filter)
try:
# Create Task
create_task_response = self.drydock_client.create_task(
design_ref=self.deckhand_design_ref,
task_action=task_action,
node_filter=self.node_filter)
except errors.ClientError as client_error:
# Dump logs from Drydock pods
self.get_k8s_logs()
raise AirflowException(client_error)
# Retrieve Task ID
self.drydock_task_id = create_task_response['task_id']
LOG.info('Drydock %s task ID is %s',
task_action, self.drydock_task_id)
# Raise Exception if we are not able to get the task_id from
# Drydock
if self.drydock_task_id:
return self.drydock_task_id
else:
raise AirflowException("Unable to create task!")
def query_task(self, interval, time_out):
# Calculate number of times to execute the 'for' loop
# Convert 'time_out' and 'interval' from string into integer
# The result from the division will be a floating number which
# We will round off to nearest whole number
end_range = round(int(time_out) / int(interval))
LOG.info('Task ID is %s', self.drydock_task_id)
# Query task status
for i in range(0, end_range + 1):
try:
# Retrieve current task state
task_state = self.drydock_client.get_task(
task_id=self.drydock_task_id)
task_status = task_state['status']
task_result = task_state['result']['status']
LOG.info("Current status of task id %s is %s",
self.drydock_task_id, task_status)
except errors.ClientError as client_error:
# Dump logs from Drydock pods
self.get_k8s_logs()
raise AirflowException(client_error)
except:
# There can be situations where there are intermittent network
# issues that prevents us from retrieving the task state. We
# will want to retry in such situations.
LOG.warning("Unable to retrieve task state. Retrying...")
# Raise Time Out Exception
if task_status == 'running' and i == end_range:
self.task_failure(False)
# Exit 'for' loop if the task is in 'complete' or 'terminated'
# state
if task_status in ['complete', 'terminated']:
LOG.info('Task result is %s', task_result)
break
else:
time.sleep(int(interval))
# Get final task result
if task_result == 'success':
LOG.info('Task id %s has been successfully completed',
self.drydock_task_id)
else:
self.task_failure(True)
def task_failure(self, _task_failure):
# Dump logs from Drydock pods
self.get_k8s_logs()
LOG.info('Retrieving all tasks records from Drydock...')
try:
# Get all tasks records
all_tasks = self.drydock_client.get_tasks()
# Create a dictionary of tasks records with 'task_id' as key
all_task_ids = {t['task_id']: t for t in all_tasks}
except errors.ClientError as client_error:
raise AirflowException(client_error)
# Retrieve the failed parent task and assign it to list
failed_task = (
[x for x in all_tasks if x['task_id'] == self.drydock_task_id])
# Print detailed information of failed parent task in json output
# Since there is only 1 failed parent task, we will print index 0
# of the list
if failed_task:
LOG.error('%s task has either failed or timed out',
failed_task[0]['action'])
LOG.error(json.dumps(failed_task[0],
indent=4,
sort_keys=True))
# Get the list of subtasks belonging to the failed parent task
subtask_id_list = failed_task[0]['subtask_id_list']
LOG.info("Printing information of failed sub-tasks...")
# Print detailed information of failed step(s) under each subtask
# This will help to provide additional information for troubleshooting
# purpose.
for subtask_id in subtask_id_list:
LOG.info("Retrieving details of subtask %s...", subtask_id)
# Retrieve task information
task = all_task_ids.get(subtask_id)
if task:
# Print subtask action and state
LOG.info("%s subtask is in %s state",
task['action'],
task['result']['status'])
# Print list containing steps in failure state
if task['result']['failures']:
LOG.error("The following steps have failed:")
LOG.error(task['result']['failures'])
message_list = (
task['result']['details']['messageList'] or [])
# Print information of failed steps
for message in message_list:
is_error = message['error'] is True
if is_error:
LOG.error(json.dumps(message,
indent=4,
sort_keys=True))
else:
LOG.info("No failed step detected for subtask %s",
subtask_id)
else:
raise AirflowException("Unable to retrieve subtask info!")
# Raise Exception to terminate workflow
if _task_failure:
raise AirflowException("Failed to Execute/Complete Task!")
else:
raise AirflowException("Task Execution Timed Out!")
class DrydockBaseOperatorPlugin(AirflowPlugin):
"""Creates DrydockBaseOperator in Airflow."""
name = 'drydock_base_operator_plugin'
operators = [DrydockBaseOperator]