Add relabel_nodes action
This adds the Shipyard workflow for the new relabel_nodes action. Related changes are https://review.openstack.org/593609 (Drydock) and https://review.openstack.org/584925 (Promenade). Change-Id: I2881199c1622e873b5fc3e948231bacd10a1beba
This commit is contained in:
parent
7fa3136470
commit
9b3cd66156
charts/shipyard
doc/source
src/bin
shipyard_airflow
shipyard_client/shipyard_client/cli
@ -377,6 +377,7 @@ conf:
|
||||
workflow_orchestrator:action_update_site: rule:admin_create
|
||||
workflow_orchestrator:action_update_software: rule:admin_create
|
||||
workflow_orchestrator:action_redeploy_server: rule:admin_create
|
||||
workflow_orchestrator:action_relabel_nodes: rule:admin_create
|
||||
paste:
|
||||
app:shipyard-api:
|
||||
paste.app_factory: shipyard_airflow.shipyard_api:paste_start_shipyard
|
||||
|
@ -231,6 +231,12 @@ N/A
|
||||
underlying implementation already provides the needed resiliency and
|
||||
redundancy.
|
||||
|
||||
relabel_nodes
|
||||
~~~~~~~~~~~~~
|
||||
|
||||
Using parameters to indicate which server(s), triggers an update to the
|
||||
Kubernetes node labels for those servers.
|
||||
|
||||
Future actions
|
||||
~~~~~~~~~~~~~~
|
||||
|
||||
|
@ -78,6 +78,15 @@ def _action_mappings():
|
||||
action_validators.validate_committed_revision,
|
||||
action_validators.validate_deployment_action_basic,
|
||||
]
|
||||
},
|
||||
'relabel_nodes': {
|
||||
'dag': 'relabel_nodes',
|
||||
'rbac_policy': policy.ACTION_RELABEL_NODES,
|
||||
'validators': [
|
||||
action_validators.validate_target_nodes,
|
||||
action_validators.validate_committed_revision,
|
||||
action_validators.validate_deployment_action_basic,
|
||||
]
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -24,6 +24,7 @@ try:
|
||||
from airflow.operators import DeploymentConfigurationOperator
|
||||
from airflow.operators import DeckhandCreateSiteActionTagOperator
|
||||
from airflow.operators import DrydockDestroyNodeOperator
|
||||
from airflow.operators import DrydockRelabelNodesOperator
|
||||
except ImportError:
|
||||
# for local testing, they are loaded from their source directory
|
||||
from shipyard_airflow.plugins.concurrency_check_operator import \
|
||||
@ -36,6 +37,8 @@ except ImportError:
|
||||
DeckhandCreateSiteActionTagOperator
|
||||
from shipyard_airflow.plugins.drydock_destroy_nodes import \
|
||||
DrydockDestroyNodeOperator
|
||||
from shipyard_airflow.plugins.drydock_relabel_nodes import \
|
||||
DrydockRelabelNodesOperator
|
||||
|
||||
try:
|
||||
# modules reside in a flat directory when deployed with dags
|
||||
@ -190,6 +193,18 @@ class CommonStepFactory(object):
|
||||
on_failure_callback=step_failure_handler,
|
||||
dag=self.dag)
|
||||
|
||||
def get_relabel_nodes(self, task_id=dn.RELABEL_NODES_DAG_NAME):
|
||||
"""Generate the relabel nodes step
|
||||
|
||||
This step uses Drydock to relabel select nodes.
|
||||
"""
|
||||
return DrydockRelabelNodesOperator(
|
||||
main_dag_name=self.parent_dag_name,
|
||||
shipyard_conf=config_path,
|
||||
task_id=task_id,
|
||||
on_failure_callback=step_failure_handler,
|
||||
dag=self.dag)
|
||||
|
||||
def get_armada_build(self, task_id=dn.ARMADA_BUILD_DAG_NAME):
|
||||
"""Generate the armada build step
|
||||
|
||||
|
@ -18,6 +18,7 @@ ARMADA_BUILD_DAG_NAME = 'armada_build'
|
||||
DESTROY_SERVER_DAG_NAME = 'destroy_server'
|
||||
DRYDOCK_BUILD_DAG_NAME = 'drydock_build'
|
||||
VALIDATE_SITE_DESIGN_DAG_NAME = 'validate_site_design'
|
||||
RELABEL_NODES_DAG_NAME = 'relabel_nodes'
|
||||
|
||||
# Steps
|
||||
ACTION_XCOM = 'action_xcom'
|
||||
|
@ -0,0 +1,66 @@
|
||||
# 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.
|
||||
from datetime import timedelta
|
||||
|
||||
import airflow
|
||||
from airflow import DAG
|
||||
|
||||
try:
|
||||
from common_step_factory import CommonStepFactory
|
||||
except ImportError:
|
||||
from shipyard_airflow.dags.common_step_factory import CommonStepFactory
|
||||
|
||||
"""relabel_nodes
|
||||
|
||||
The top-level orchestration DAG for updating only the node labels
|
||||
using the Undercloud platform.
|
||||
"""
|
||||
|
||||
PARENT_DAG_NAME = 'relabel_nodes'
|
||||
|
||||
default_args = {
|
||||
'owner': 'airflow',
|
||||
'depends_on_past': False,
|
||||
'start_date': airflow.utils.dates.days_ago(1),
|
||||
'email': [''],
|
||||
'email_on_failure': False,
|
||||
'email_on_retry': False,
|
||||
'provide_context': True,
|
||||
'retries': 0,
|
||||
'retry_delay': timedelta(seconds=30),
|
||||
}
|
||||
|
||||
dag = DAG(PARENT_DAG_NAME, default_args=default_args, schedule_interval=None)
|
||||
|
||||
step_factory = CommonStepFactory(parent_dag_name=PARENT_DAG_NAME,
|
||||
dag=dag,
|
||||
default_args=default_args,
|
||||
action_type='targeted')
|
||||
|
||||
# create steps
|
||||
action_xcom = step_factory.get_action_xcom()
|
||||
concurrency_check = step_factory.get_concurrency_check()
|
||||
preflight = step_factory.get_preflight()
|
||||
get_rendered_doc = step_factory.get_get_rendered_doc()
|
||||
deployment_configuration = step_factory.get_deployment_configuration()
|
||||
validate_site_design = step_factory.get_validate_site_design()
|
||||
relabel_nodes = step_factory.get_relabel_nodes()
|
||||
|
||||
# DAG Wiring
|
||||
concurrency_check.set_upstream(action_xcom)
|
||||
preflight.set_upstream(concurrency_check)
|
||||
get_rendered_doc.set_upstream(preflight)
|
||||
deployment_configuration.set_upstream(get_rendered_doc)
|
||||
validate_site_design.set_upstream(deployment_configuration)
|
||||
relabel_nodes.set_upstream(validate_site_design)
|
@ -29,7 +29,7 @@ AIRFLOW_DB = 'airflows_own_db'
|
||||
|
||||
# each set in this list of sets indicates DAGs that shouldn't execute together
|
||||
CONFLICTING_DAG_SETS = [set(['deploy_site', 'update_site', 'update_software',
|
||||
'redeploy_server'])]
|
||||
'redeploy_server', 'relabel_nodes'])]
|
||||
|
||||
|
||||
def find_conflicting_dag_set(dag_name, conflicting_dag_sets=None):
|
||||
|
@ -57,6 +57,8 @@ class DeploymentConfigurationOperator(BaseOperator):
|
||||
"physical_provisioner.prepare_site_timeout": 300,
|
||||
"physical_provisioner.verify_interval": 10,
|
||||
"physical_provisioner.verify_timeout": 60,
|
||||
"physical_provisioner.relabel_nodes_interval": 30,
|
||||
"physical_provisioner.relabel_nodes_timeout": 900,
|
||||
"kubernetes.node_status_interval": 30,
|
||||
"kubernetes.node_status_timeout": 1800,
|
||||
"kubernetes_provisioner.drain_timeout": 3600,
|
||||
|
@ -0,0 +1,110 @@
|
||||
# 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.
|
||||
"""Update node labels using Drydock
|
||||
"""
|
||||
import logging
|
||||
|
||||
from airflow.exceptions import AirflowException
|
||||
from airflow.plugins_manager import AirflowPlugin
|
||||
|
||||
try:
|
||||
from drydock_base_operator import DrydockBaseOperator
|
||||
from drydock_base_operator import gen_node_name_filter
|
||||
from drydock_errors import (
|
||||
DrydockTaskFailedException,
|
||||
DrydockTaskTimeoutException
|
||||
)
|
||||
except ImportError:
|
||||
from shipyard_airflow.plugins.drydock_base_operator import \
|
||||
DrydockBaseOperator
|
||||
from shipyard_airflow.plugins.drydock_base_operator import \
|
||||
gen_node_name_filter
|
||||
from shipyard_airflow.plugins.drydock_errors import (
|
||||
DrydockTaskFailedException,
|
||||
DrydockTaskTimeoutException
|
||||
)
|
||||
|
||||
LOG = logging.getLogger(__name__)
|
||||
|
||||
|
||||
class DrydockRelabelNodesOperator(DrydockBaseOperator):
|
||||
"""Drydock Relabel Nodes Operator
|
||||
|
||||
Update Kubernetes node labels on a targeted set of nodes
|
||||
using Drydock.
|
||||
"""
|
||||
|
||||
def do_execute(self):
|
||||
self.successes = []
|
||||
|
||||
LOG.info("Relabeling nodes [%s]", ", ".join(self.target_nodes))
|
||||
self.setup_configured_values()
|
||||
self.node_filter = gen_node_name_filter(self.target_nodes)
|
||||
self.execute_relabel()
|
||||
self.successes = self.get_successes_for_task(self.drydock_task_id)
|
||||
self.report_summary()
|
||||
if not self.is_task_successful():
|
||||
raise AirflowException(
|
||||
"One or more nodes requested for relabeling failed to relabel"
|
||||
)
|
||||
|
||||
def setup_configured_values(self):
|
||||
"""Retrieve and localize the interval and timeout values for destroy
|
||||
"""
|
||||
self.q_interval = \
|
||||
self.dc['physical_provisioner.relabel_nodes_interval']
|
||||
self.task_timeout = \
|
||||
self.dc['physical_provisioner.relabel_nodes_timeout']
|
||||
|
||||
def execute_relabel(self):
|
||||
# Trigger DryDock to execute task
|
||||
task_name = 'relabel_nodes'
|
||||
|
||||
self.create_task(task_name)
|
||||
|
||||
# Query Task
|
||||
try:
|
||||
self.query_task(self.q_interval, self.task_timeout)
|
||||
except DrydockTaskFailedException:
|
||||
LOG.exception("Task %s has failed. Some nodes may have been "
|
||||
"relabeled. The report at the end of processing "
|
||||
"this step contains the results", task_name)
|
||||
except DrydockTaskTimeoutException:
|
||||
LOG.warn("Task %s has timed out after %s seconds. Some nodes may "
|
||||
"have been relabeled. The report at the end of "
|
||||
"processing this step contains the results", task_name,
|
||||
self.task_timeout)
|
||||
|
||||
def report_summary(self):
|
||||
"""Reports the successfully relabeled nodes"""
|
||||
failed = list(set(self.target_nodes) - set(self.successes))
|
||||
LOG.info("===== Relabel Nodes Summary =====")
|
||||
LOG.info(" Nodes requested: %s", ", ".join(sorted(self.target_nodes)))
|
||||
LOG.info(" Nodes relabeled: %s ", ", ".join(sorted(self.successes)))
|
||||
LOG.info(" Nodes not relabeled: %s", ", ".join(sorted(failed)))
|
||||
LOG.info("===== End Relabel Nodes Summary =====")
|
||||
|
||||
# TODO: redundant with drydock_destroy_nodes...worth refactoring?
|
||||
def is_task_successful(self):
|
||||
"""Boolean if the task was completely succesful."""
|
||||
failed = set(self.target_nodes) - set(self.successes)
|
||||
return not failed
|
||||
|
||||
|
||||
class DrydockRelabelNodesOperatorPlugin(AirflowPlugin):
|
||||
|
||||
"""Creates DrydockRelabelNodesOperator in Airflow."""
|
||||
|
||||
name = 'drydock_relabel_nodes_operator'
|
||||
operators = [DrydockRelabelNodesOperator]
|
@ -45,6 +45,7 @@ ACTION_DEPLOY_SITE = 'workflow_orchestrator:action_deploy_site'
|
||||
ACTION_UPDATE_SITE = 'workflow_orchestrator:action_update_site'
|
||||
ACTION_UPDATE_SOFTWARE = 'workflow_orchestrator:action_update_software'
|
||||
ACTION_REDEPLOY_SERVER = 'workflow_orchestrator:action_redeploy_server'
|
||||
ACTION_RELABEL_NODES = 'workflow_orchestrator:action_relabel_nodes'
|
||||
|
||||
|
||||
class ShipyardPolicy(object):
|
||||
@ -252,6 +253,15 @@ class ShipyardPolicy(object):
|
||||
'method': 'POST'
|
||||
}]
|
||||
),
|
||||
policy.DocumentedRuleDefault(
|
||||
ACTION_RELABEL_NODES,
|
||||
RULE_ADMIN_REQUIRED,
|
||||
'Create a workflow action to relabel target nodes',
|
||||
[{
|
||||
'path': '/api/v1.0/actions',
|
||||
'method': 'POST'
|
||||
}]
|
||||
),
|
||||
]
|
||||
|
||||
# Regions Policy
|
||||
|
@ -37,6 +37,10 @@ data:
|
||||
type: 'integer'
|
||||
verify_timeout:
|
||||
type: 'integer'
|
||||
relabel_nodes_interval:
|
||||
type: 'integer'
|
||||
relabel_nodes_timeout:
|
||||
type: 'integer'
|
||||
additionalProperties: false
|
||||
kubernetes:
|
||||
type: 'object'
|
||||
|
@ -0,0 +1,217 @@
|
||||
# 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.
|
||||
"""Tests for drydock_relabel_nodes operator functions"""
|
||||
import os
|
||||
from unittest import mock
|
||||
|
||||
from airflow.exceptions import AirflowException
|
||||
import pytest
|
||||
|
||||
from shipyard_airflow.plugins.drydock_relabel_nodes import \
|
||||
DrydockRelabelNodesOperator
|
||||
from shipyard_airflow.plugins.drydock_errors import (
|
||||
DrydockTaskFailedException,
|
||||
DrydockTaskTimeoutException
|
||||
)
|
||||
|
||||
|
||||
CONF_FILE = os.path.join(os.path.dirname(__file__), 'test.conf')
|
||||
ALL_SUCCESES = ['node1', 'node2', 'node3']
|
||||
|
||||
# The top level result should have all successes specified
|
||||
TASK_DICT = {
|
||||
'0': {
|
||||
'result': {
|
||||
'successes': ['node1', 'node2', 'node3'],
|
||||
'status': 'success',
|
||||
},
|
||||
'subtask_id_list': ['1'],
|
||||
'status': 'complete'
|
||||
},
|
||||
'1': {
|
||||
'result': {
|
||||
'successes': ['node3'],
|
||||
'status': 'success',
|
||||
},
|
||||
'subtask_id_list': ['2', '3'],
|
||||
'status': 'complete'
|
||||
},
|
||||
}
|
||||
|
||||
|
||||
def _fake_get_task_dict(task_id):
|
||||
return TASK_DICT[task_id]
|
||||
|
||||
|
||||
class TestDrydockRelabelNodesOperator:
|
||||
def test_setup_configured_values(self):
|
||||
op = DrydockRelabelNodesOperator(main_dag_name="main",
|
||||
shipyard_conf=CONF_FILE,
|
||||
task_id="t1")
|
||||
op.dc = {
|
||||
'physical_provisioner.relabel_nodes_interval': 1,
|
||||
'physical_provisioner.relabel_nodes_timeout': 2,
|
||||
}
|
||||
op.setup_configured_values()
|
||||
assert op.q_interval == 1
|
||||
assert op.task_timeout == 2
|
||||
|
||||
def test_success_functions(self, caplog):
|
||||
op = DrydockRelabelNodesOperator(main_dag_name="main",
|
||||
shipyard_conf=CONF_FILE,
|
||||
task_id="t1")
|
||||
# testing with lists and sets.
|
||||
op.target_nodes = ['n0', 'n1', 'n2']
|
||||
op.successes = ['n1']
|
||||
caplog.clear()
|
||||
op.report_summary()
|
||||
assert " Nodes requested: n0, n1, n2" in caplog.text
|
||||
assert " Nodes relabeled: n1" in caplog.text
|
||||
assert " Nodes not relabeled: n0, n2" in caplog.text
|
||||
assert not op.is_task_successful()
|
||||
|
||||
op.target_nodes = set(['n0', 'n1', 'n2'])
|
||||
op.successes = []
|
||||
caplog.clear()
|
||||
op.report_summary()
|
||||
assert " Nodes requested: n0, n1, n2" in caplog.text
|
||||
assert " Nodes not relabeled: n0, n1, n2" in caplog.text
|
||||
assert not op.is_task_successful()
|
||||
|
||||
op.target_nodes = set(['n0', 'n1', 'n2'])
|
||||
op.successes = set(['n0', 'n1', 'n2'])
|
||||
caplog.clear()
|
||||
op.report_summary()
|
||||
assert " Nodes requested: n0, n1, n2" in caplog.text
|
||||
assert " Nodes relabeled: n0, n1, n2" in caplog.text
|
||||
assert op.is_task_successful()
|
||||
|
||||
@mock.patch.object(
|
||||
DrydockRelabelNodesOperator, 'create_task'
|
||||
)
|
||||
@mock.patch.object(
|
||||
DrydockRelabelNodesOperator, 'query_task'
|
||||
)
|
||||
def test_execute_relabel_nodes_simple_success(self, qt, ct, caplog):
|
||||
op = DrydockRelabelNodesOperator(main_dag_name="main",
|
||||
shipyard_conf=CONF_FILE,
|
||||
task_id="t1")
|
||||
op.dc = {
|
||||
'physical_provisioner.relabel_nodes_interval': 1,
|
||||
'physical_provisioner.relabel_nodes_timeout': 2,
|
||||
}
|
||||
op.setup_configured_values()
|
||||
op.execute_relabel()
|
||||
assert qt.called
|
||||
assert ct.called
|
||||
assert not caplog.records
|
||||
|
||||
@mock.patch.object(
|
||||
DrydockRelabelNodesOperator, 'create_task'
|
||||
)
|
||||
@mock.patch.object(
|
||||
DrydockRelabelNodesOperator, 'query_task',
|
||||
side_effect=DrydockTaskFailedException("test")
|
||||
)
|
||||
def test_execute_relabel_nodes_query_fail(self, qt, ct, caplog):
|
||||
op = DrydockRelabelNodesOperator(main_dag_name="main",
|
||||
shipyard_conf=CONF_FILE,
|
||||
task_id="t1")
|
||||
op.dc = {
|
||||
'physical_provisioner.relabel_nodes_interval': 1,
|
||||
'physical_provisioner.relabel_nodes_timeout': 2,
|
||||
}
|
||||
op.setup_configured_values()
|
||||
op.execute_relabel()
|
||||
assert qt.called
|
||||
assert ct.called
|
||||
assert "Task relabel_nodes has failed." in caplog.text
|
||||
|
||||
@mock.patch.object(
|
||||
DrydockRelabelNodesOperator, 'create_task'
|
||||
)
|
||||
@mock.patch.object(
|
||||
DrydockRelabelNodesOperator, 'query_task',
|
||||
side_effect=DrydockTaskTimeoutException("test")
|
||||
)
|
||||
def test_execute_relabel_nodes_query_timeout(self, qt, ct, caplog):
|
||||
op = DrydockRelabelNodesOperator(main_dag_name="main",
|
||||
shipyard_conf=CONF_FILE,
|
||||
task_id="t1")
|
||||
op.dc = {
|
||||
'physical_provisioner.relabel_nodes_interval': 1,
|
||||
'physical_provisioner.relabel_nodes_timeout': 2,
|
||||
}
|
||||
op.setup_configured_values()
|
||||
op.execute_relabel()
|
||||
assert qt.called
|
||||
assert ct.called
|
||||
assert "Task relabel_nodes has timed out after 2 seconds." in (
|
||||
caplog.text)
|
||||
|
||||
@mock.patch.object(
|
||||
DrydockRelabelNodesOperator, 'get_successes_for_task',
|
||||
return_value=['n0', 'n1']
|
||||
)
|
||||
@mock.patch.object(
|
||||
DrydockRelabelNodesOperator, 'create_task'
|
||||
)
|
||||
@mock.patch.object(
|
||||
DrydockRelabelNodesOperator, 'query_task',
|
||||
side_effect=DrydockTaskTimeoutException("test")
|
||||
)
|
||||
def test_do_execute_fail(self, qt, ct, gs, caplog):
|
||||
op = DrydockRelabelNodesOperator(main_dag_name="main",
|
||||
shipyard_conf=CONF_FILE,
|
||||
task_id="t1")
|
||||
op.dc = {
|
||||
'physical_provisioner.relabel_nodes_interval': 1,
|
||||
'physical_provisioner.relabel_nodes_timeout': 2,
|
||||
}
|
||||
op.target_nodes = ['n0', 'n1', 'n2']
|
||||
with pytest.raises(AirflowException) as ae:
|
||||
op.do_execute()
|
||||
assert qt.called
|
||||
assert ct.called
|
||||
assert gs.called
|
||||
assert "Task relabel_nodes has timed out after 2 seconds." in (
|
||||
caplog.text)
|
||||
assert ("One or more nodes requested for relabeling failed to "
|
||||
"relabel") in str(ae.value)
|
||||
|
||||
@mock.patch.object(
|
||||
DrydockRelabelNodesOperator, 'get_successes_for_task',
|
||||
return_value=['n0', 'n1', 'n2']
|
||||
)
|
||||
@mock.patch.object(
|
||||
DrydockRelabelNodesOperator, 'create_task'
|
||||
)
|
||||
@mock.patch.object(
|
||||
DrydockRelabelNodesOperator, 'query_task',
|
||||
)
|
||||
def test_do_execute(self, qt, ct, gs, caplog):
|
||||
op = DrydockRelabelNodesOperator(main_dag_name="main",
|
||||
shipyard_conf=CONF_FILE,
|
||||
task_id="t1")
|
||||
op.dc = {
|
||||
'physical_provisioner.relabel_nodes_interval': 1,
|
||||
'physical_provisioner.relabel_nodes_timeout': 2,
|
||||
}
|
||||
op.target_nodes = ['n0', 'n1', 'n2']
|
||||
op.do_execute()
|
||||
assert qt.called
|
||||
assert ct.called
|
||||
assert gs.called
|
||||
assert " Nodes requested: n0, n1, n2" in caplog.text
|
||||
assert " Nodes relabeled: n0, n1, n2" in caplog.text
|
@ -46,6 +46,9 @@ update_software: Starts an update that only exercises the software portion of
|
||||
redeploy_server: Using parameters to indicate which server(s), triggers a
|
||||
redeployment of servers to the last committed design and
|
||||
secrets.
|
||||
|
||||
relabel_nodes: Using parameters to indicate which server(s), updates the
|
||||
labels for those servers.
|
||||
'''
|
||||
|
||||
|
||||
|
@ -19,7 +19,7 @@ from arrow.parser import ParserError
|
||||
def check_action_command(ctx, action_command):
|
||||
"""Verifies the action command is valid"""
|
||||
valid_commands = ['deploy_site', 'update_site', 'update_software',
|
||||
'redeploy_server']
|
||||
'redeploy_server', 'relabel_nodes']
|
||||
if action_command not in valid_commands:
|
||||
ctx.fail('Invalid action command. The action commands available are '
|
||||
' {}'.format(', '.join(valid_commands)))
|
||||
|
Loading…
x
Reference in New Issue
Block a user