From d229d52292626d93d2f6c2c6b3942dd9c38a289b Mon Sep 17 00:00:00 2001 From: Sean Eagan Date: Tue, 28 Aug 2018 11:55:51 -0500 Subject: [PATCH] Parallelize unsequenced chart group deployments This changes unsequenced chart group deployments, such that each chart in the group is deployed in parallel, including the install/upgrade, wait, and tests. Previously, whether and when to wait was entangled with whether or not the chart group was sequenced, since running helm install/upgrade's native wait (which cannot be run later) and armada's labels based wait, delayed (or even prevented in the case of failure) the next chart from being deployed, which is the intention for sequenced, but not for unsequenced. With this patchset, sequencing and waiting are now orthogonal. Hence we can now allow the user to explictly specify whether to wait, which this patchset does for the case of helm's native wait via a new `wait.native.enabled` flag, which defaults to true. Previously, armada's labels-based wait sometimes occurred both between charts and at the end of the chart group. It now occurs once directly after chart deployment. Previously, passing armada's --wait was documented to be equivalent to forcing sequencing of chart groups, however helm tests did not run in sequence as they normally would with sequenced chart groups, they now do. Since chart deploys can now occur in parallel, log messages for each become interleaved, and thus when armada is deploying a chart, log messages are updated to contain identifying information about which chart deployment they are for. Change-Id: I9d13245c40887712333aaccfb044dcdc4b83988e --- armada/conf/__init__.py | 41 +- armada/exceptions/armada_exceptions.py | 10 + armada/handlers/armada.py | 398 +++--------------- armada/handlers/chart_deploy.py | 290 +++++++++++++ armada/schemas/armada-chart-schema.yaml | 8 + armada/tests/test_utils.py | 25 +- armada/tests/unit/handlers/test_armada.py | 109 ++--- .../operations/guide-build-armada-yaml.rst | 41 +- 8 files changed, 501 insertions(+), 421 deletions(-) create mode 100644 armada/handlers/chart_deploy.py diff --git a/armada/conf/__init__.py b/armada/conf/__init__.py index 8fbcbdd5..b6d988e3 100644 --- a/armada/conf/__init__.py +++ b/armada/conf/__init__.py @@ -12,7 +12,9 @@ # See the License for the specific language governing permissions and # limitations under the License. +import logging import os +import threading from oslo_config import cfg from oslo_log import log @@ -42,10 +44,27 @@ def set_app_default_configs(): config_files = _get_config_files() if all([os.path.exists(x) for x in config_files]): CONF([], project='armada', default_config_files=config_files) + setup_chart_deploy_aware_logging() set_default_for_default_log_levels() default.register_opts(CONF) +# Stores chart being deployed (if any) in current thread. +current_chart_thread_local = threading.local() + + +def get_current_chart(): + return getattr(current_chart_thread_local, 'chart', None) + + +def set_current_chart(chart): + current_chart_thread_local.chart = chart + + +def setup_chart_deploy_aware_logging(): + logging.setLoggerClass(ChartDeployAwareLogger) + + def set_default_for_default_log_levels(): """Set the default for the default_log_levels option for Armada. Armada uses some packages that other OpenStack services don't use that do @@ -56,5 +75,23 @@ def set_default_for_default_log_levels(): extra_log_level_defaults = ['kubernetes.client.rest=INFO'] - log.set_defaults(default_log_levels=log.get_default_log_levels() + - extra_log_level_defaults) + log.set_defaults( + default_log_levels=log.get_default_log_levels() + + extra_log_level_defaults, ) + + +class ChartDeployAwareLogger(logging.Logger): + """Includes name of chart currently being deployed (if any) in log + messages. + """ + + def _log(self, level, msg, *args, **kwargs): + chart = get_current_chart() + if chart: + name = chart['chart_name'] + prefix = '[chart={}]: '.format(name) + else: + prefix = '' + prefixed = '{}{}'.format(prefix, msg) + return super(ChartDeployAwareLogger, self)._log( + level, prefixed, *args, **kwargs) diff --git a/armada/exceptions/armada_exceptions.py b/armada/exceptions/armada_exceptions.py index c2f996f7..b68f5033 100644 --- a/armada/exceptions/armada_exceptions.py +++ b/armada/exceptions/armada_exceptions.py @@ -77,3 +77,13 @@ class InvalidWaitTypeException(ArmadaException): self._message = ( 'Armada encountered invalid wait type: %s' % wait_type) super(InvalidWaitTypeException, self).__init__(self._message) + + +class ChartDeployException(ArmadaException): + ''' + Exception that occurs while deploying charts. + ''' + + def __init__(self, chart_names): + self._message = ('Exception deploying charts: %s' % chart_names) + super(ChartDeployException, self).__init__(self._message) diff --git a/armada/handlers/armada.py b/armada/handlers/armada.py index ef8fb14a..897c44a0 100644 --- a/armada/handlers/armada.py +++ b/armada/handlers/armada.py @@ -12,26 +12,21 @@ # See the License for the specific language governing permissions and # limitations under the License. -import functools -import time -import yaml - +from concurrent.futures import ThreadPoolExecutor, as_completed from oslo_config import cfg from oslo_log import log as logging from armada import const +from armada.conf import set_current_chart from armada.exceptions import armada_exceptions from armada.exceptions import override_exceptions from armada.exceptions import source_exceptions from armada.exceptions import tiller_exceptions from armada.exceptions import validate_exceptions -from armada.handlers.chartbuilder import ChartBuilder +from armada.handlers.chart_deploy import ChartDeploy from armada.handlers.manifest import Manifest from armada.handlers.override import Override -from armada.handlers.release_diff import ReleaseDiff -from armada.handlers.test import test_release_for_success from armada.handlers.tiller import Tiller -from armada.handlers.wait import get_wait_for from armada.utils.release import release_prefixer from armada.utils import source @@ -89,12 +84,9 @@ class Armada(object): tiller_port = tiller_port or CONF.tiller_port tiller_namespace = tiller_namespace or CONF.tiller_namespace - self.disable_update_pre = disable_update_pre - self.disable_update_post = disable_update_post self.enable_chart_cleanup = enable_chart_cleanup self.dry_run = dry_run self.force_wait = force_wait - self.timeout = timeout # TODO: Use dependency injection i.e. pass in a Tiller instead of # creating it here. self.tiller = Tiller( @@ -109,19 +101,12 @@ class Armada(object): except (validate_exceptions.InvalidManifestException, override_exceptions.InvalidOverrideValueException): raise - self.k8s_wait_attempts = k8s_wait_attempts - self.k8s_wait_attempt_sleep = k8s_wait_attempt_sleep self.manifest = Manifest( self.documents, target_manifest=target_manifest).get_manifest() self.cloned_dirs = set() - - def find_release_chart(self, known_releases, release_name): - ''' - Find a release given a list of known_releases and a release name - ''' - for release, _, chart, values, _ in known_releases: - if release == release_name: - return chart, values + self.chart_deploy = ChartDeploy( + disable_update_pre, disable_update_post, self.dry_run, + k8s_wait_attempts, k8s_wait_attempt_sleep, timeout, self.tiller) def pre_flight_ops(self): """Perform a series of checks and operations to ensure proper @@ -240,9 +225,12 @@ class Armada(object): for chartgroup in manifest_data.get(const.KEYWORD_GROUPS, []): cg_name = chartgroup.get('name', '') cg_desc = chartgroup.get('description', '') - cg_sequenced = chartgroup.get('sequenced', False) - LOG.info('Processing ChartGroup: %s (%s), sequenced=%s', cg_name, - cg_desc, cg_sequenced) + cg_sequenced = chartgroup.get('sequenced', + False) or self.force_wait + + LOG.info('Processing ChartGroup: %s (%s), sequenced=%s%s', cg_name, + cg_desc, cg_sequenced, + ' (forced)' if self.force_wait else '') # TODO(MarshM): Deprecate the `test_charts` key cg_test_all_charts = chartgroup.get('test_charts') @@ -255,284 +243,61 @@ class Armada(object): # explicitly disable helm tests if they choose cg_test_all_charts = True - ns_label_set = set() - tests_to_run = [] - cg_charts = chartgroup.get(const.KEYWORD_CHARTS, []) + charts = map(lambda x: x.get('chart', {}), cg_charts) - # Track largest Chart timeout to stop the ChartGroup at the end - cg_max_timeout = 0 + def deploy_chart(chart): + set_current_chart(chart) + try: + return self.chart_deploy.execute(chart, cg_test_all_charts, + prefix, deployed_releases, + failed_releases) + finally: + set_current_chart(None) - for chart_entry in cg_charts: - chart = chart_entry.get('chart', {}) - namespace = chart.get('namespace') - release = chart.get('release') - release_name = release_prefixer(prefix, release) - LOG.info('Processing Chart, release=%s', release_name) + results = [] + failures = [] - values = chart.get('values', {}) - pre_actions = {} - post_actions = {} - - protected = chart.get('protected', {}) - p_continue = protected.get('continue_processing', False) - - # Check for existing FAILED release, and purge - if release_name in [rel[0] for rel in failed_releases]: - LOG.info('Purging FAILED release %s before deployment.', - release_name) - if protected: - if p_continue: - LOG.warn( - 'Release %s is `protected`, ' - 'continue_processing=True. Operator must ' - 'handle FAILED release manually.', - release_name) - msg['protected'].append(release_name) - continue - else: - LOG.error( - 'Release %s is `protected`, ' - 'continue_processing=False.', release_name) - raise armada_exceptions.ProtectedReleaseException( - release_name) - else: - # Purge the release - self.tiller.uninstall_release(release_name) - msg['purge'].append(release_name) - - # NOTE(MarshM): Calculating `wait_timeout` is unfortunately - # overly complex. The order of precedence is currently: - # 1) User provided override via API/CLI (default 0 if not - # provided by client/user). - # 2) Chart's `data.wait.timeout`, or... - # 3) Chart's `data.timeout` (deprecated). - # 4) const.DEFAULT_CHART_TIMEOUT, if nothing is ever - # specified, for use in waiting for final ChartGroup - # health and helm tests, but ignored for the actual - # install/upgrade of the Chart. - # NOTE(MarshM): Not defining a timeout has a side effect of - # allowing Armada to install charts with a circular - # dependency defined between components. - - # TODO(MarshM): Deprecated, remove the following block - deprecated_timeout = chart.get('timeout', None) - if isinstance(deprecated_timeout, int): - LOG.warn('The `timeout` key is deprecated and support ' - 'for this will be removed soon. Use ' - '`wait.timeout` instead.') - - wait_values = chart.get('wait', {}) - wait_labels = wait_values.get('labels', {}) - wait_timeout = self.timeout - if wait_timeout <= 0: - wait_timeout = wait_values.get('timeout', wait_timeout) - # TODO(MarshM): Deprecated, remove the following check - if wait_timeout <= 0: - wait_timeout = deprecated_timeout or wait_timeout - - # Determine wait logic - # NOTE(Dan Kim): Conditions to wait are below : - # 1) set sequenced=True in chart group - # 2) set force_wait param - # 3) add Chart's `data.wait.timeout` - # --timeout param will do not set wait=True, it just change - # max timeout of chart's deployment. (default: 900) - this_chart_should_wait = (cg_sequenced or self.force_wait or - (bool(wait_values) and - (wait_timeout > 0))) - - # If there is still no timeout, we need to use a default - # (item 4 in note above) - if wait_timeout <= 0: - LOG.warn('No Chart timeout specified, using default: %ss', - const.DEFAULT_CHART_TIMEOUT) - wait_timeout = const.DEFAULT_CHART_TIMEOUT - - # Naively take largest timeout to apply at end - # TODO(MarshM) better handling of timeout/timer - cg_max_timeout = max(wait_timeout, cg_max_timeout) - - test_chart_override = chart.get('test') - # Use old default value when not using newer `test` key - test_cleanup = True - if test_chart_override is None: - test_this_chart = cg_test_all_charts - elif isinstance(test_chart_override, bool): - LOG.warn('Boolean value for chart `test` key is' - ' deprecated and support for this will' - ' be removed. Use `test.enabled` ' - 'instead.') - test_this_chart = test_chart_override + # Returns whether or not there was a failure + def handle_result(chart, get_result): + name = chart['chart_name'] + try: + result = get_result() + except Exception as e: + LOG.error('Chart deploy [%s] failed: %s', name, e) + failures.append(name) + return True else: - # NOTE: helm tests are enabled by default - test_this_chart = test_chart_override.get('enabled', True) - test_cleanup = test_chart_override.get('options', {}).get( - 'cleanup', False) + results.append(result) + return False - chartbuilder = ChartBuilder(chart) - new_chart = chartbuilder.get_helm_chart() + if cg_sequenced: + for chart in charts: + if (handle_result(chart, lambda: deploy_chart(chart))): + break + else: + with ThreadPoolExecutor( + max_workers=len(cg_charts)) as executor: + future_to_chart = { + executor.submit(deploy_chart, chart): chart + for chart in charts + } - # Begin Chart timeout deadline - deadline = time.time() + wait_timeout + for future in as_completed(future_to_chart): + chart = future_to_chart[future] + handle_result(chart, future.result) - # TODO(mark-burnett): It may be more robust to directly call - # tiller status to decide whether to install/upgrade rather - # than checking for list membership. - if release_name in [rel[0] for rel in deployed_releases]: + if failures: + LOG.error('Chart deploy(s) failed: %s', failures) + raise armada_exceptions.ChartDeployException(failures) - # indicate to the end user what path we are taking - LOG.info("Upgrading release %s in namespace %s", - release_name, namespace) - # extract the installed chart and installed values from the - # latest release so we can compare to the intended state - old_chart, old_values_string = self.find_release_chart( - deployed_releases, release_name) - - upgrade = chart.get('upgrade', {}) - disable_hooks = upgrade.get('no_hooks', False) - force = upgrade.get('force', False) - recreate_pods = upgrade.get('recreate_pods', False) - - LOG.info("Checking Pre/Post Actions") - if upgrade: - upgrade_pre = upgrade.get('pre', {}) - upgrade_post = upgrade.get('post', {}) - - if not self.disable_update_pre and upgrade_pre: - pre_actions = upgrade_pre - - if not self.disable_update_post and upgrade_post: - post_actions = upgrade_post - - try: - old_values = yaml.safe_load(old_values_string) - except yaml.YAMLError: - chart_desc = '{} (previously deployed)'.format( - old_chart.metadata.name) - raise armada_exceptions.\ - InvalidOverrideValuesYamlException(chart_desc) - - LOG.info('Checking for updates to chart release inputs.') - diff = self.get_diff(old_chart, old_values, new_chart, - values) - - if not diff: - LOG.info("Found no updates to chart release inputs") - continue - - LOG.info("Found updates to chart release inputs") - LOG.debug("%s", diff) - msg['diff'].append({chart['release']: str(diff)}) - - # TODO(MarshM): Add tiller dry-run before upgrade and - # consider deadline impacts - - # do actual update - timer = int(round(deadline - time.time())) - LOG.info('Beginning Upgrade, wait=%s, timeout=%ss', - this_chart_should_wait, timer) - tiller_result = self.tiller.update_release( - new_chart, - release_name, - namespace, - pre_actions=pre_actions, - post_actions=post_actions, - disable_hooks=disable_hooks, - values=yaml.safe_dump(values), - wait=this_chart_should_wait, - timeout=timer, - force=force, - recreate_pods=recreate_pods) - - if this_chart_should_wait: - self._wait_until_ready(release_name, wait_labels, - namespace, timer) - - # Track namespace+labels touched by upgrade - ns_label_set.add((namespace, tuple(wait_labels.items()))) - - LOG.info('Upgrade completed with results from Tiller: %s', - tiller_result.__dict__) - msg['upgrade'].append(release_name) - - # process install - else: - LOG.info("Installing release %s in namespace %s", - release_name, namespace) - - timer = int(round(deadline - time.time())) - LOG.info('Beginning Install, wait=%s, timeout=%ss', - this_chart_should_wait, timer) - tiller_result = self.tiller.install_release( - new_chart, - release_name, - namespace, - values=yaml.safe_dump(values), - wait=this_chart_should_wait, - timeout=timer) - - if this_chart_should_wait: - self._wait_until_ready(release_name, wait_labels, - namespace, timer) - - # Track namespace+labels touched by install - ns_label_set.add((namespace, tuple(wait_labels.items()))) - - LOG.info('Install completed with results from Tiller: %s', - tiller_result.__dict__) - msg['install'].append(release_name) - - # Keeping track of time remaining - timer = int(round(deadline - time.time())) - test_chart_args = (release_name, timer, test_cleanup) - if test_this_chart: - # Sequenced ChartGroup should run tests after each Chart - if cg_sequenced: - LOG.info( - 'Running sequenced test, timeout remaining: ' - '%ss.', timer) - self._test_chart(*test_chart_args) - - # Un-sequenced ChartGroup should run tests at the end - else: - tests_to_run.append( - functools.partial(self._test_chart, - *test_chart_args)) + for result in results: + for k, v in result.items(): + msg[k].append(v) # End of Charts in ChartGroup LOG.info('All Charts applied in ChartGroup %s.', cg_name) - # After all Charts are applied, we should wait for the entire - # ChartGroup to become healthy by looking at the namespaces seen - # TODO(MarshM): Need to determine a better timeout - # (not cg_max_timeout) - if cg_max_timeout <= 0: - cg_max_timeout = const.DEFAULT_CHART_TIMEOUT - deadline = time.time() + cg_max_timeout - for (ns, labels) in ns_label_set: - labels_dict = dict(labels) - timer = int(round(deadline - time.time())) - LOG.info( - 'Final ChartGroup wait for healthy namespace=%s, ' - 'labels=(%s), timeout remaining: %ss.', ns, labels_dict, - timer) - if timer <= 0: - reason = ('Timeout expired waiting on namespace: %s, ' - 'labels: (%s)' % (ns, labels_dict)) - LOG.error(reason) - raise armada_exceptions.ArmadaTimeoutException(reason) - - self._wait_until_ready( - release_name=None, - wait_labels=labels_dict, - namespace=ns, - timeout=timer) - - # After entire ChartGroup is healthy, run any pending tests - for callback in tests_to_run: - callback() - self.post_flight_ops() if self.enable_chart_cleanup: @@ -554,58 +319,6 @@ class Armada(object): LOG.debug('Removing cloned temp directory: %s', cloned_dir) source.source_cleanup(cloned_dir) - def _wait_until_ready(self, release_name, wait_labels, namespace, timeout): - if self.dry_run: - LOG.info( - 'Skipping wait during `dry-run`, would have waited on ' - 'namespace=%s, labels=(%s) for %ss.', namespace, wait_labels, - timeout) - return - - LOG.info('Waiting for release=%s', release_name) - - waits = [ - get_wait_for('job', self.tiller.k8s, skip_if_none_found=True), - get_wait_for('pod', self.tiller.k8s) - ] - deadline = time.time() + timeout - deadline_remaining = timeout - for wait in waits: - if deadline_remaining <= 0: - reason = ( - 'Timeout expired waiting for release=%s' % release_name) - LOG.error(reason) - raise armada_exceptions.ArmadaTimeoutException(reason) - - wait.wait( - labels=wait_labels, - namespace=namespace, - k8s_wait_attempts=self.k8s_wait_attempts, - k8s_wait_attempt_sleep=self.k8s_wait_attempt_sleep, - timeout=timeout) - deadline_remaining = int(round(deadline - time.time())) - - def _test_chart(self, release_name, timeout, cleanup): - if self.dry_run: - LOG.info( - 'Skipping test during `dry-run`, would have tested ' - 'release=%s with timeout %ss.', release_name, timeout) - return True - - if timeout <= 0: - reason = ('Timeout expired before testing ' - 'release %s' % release_name) - LOG.error(reason) - raise armada_exceptions.ArmadaTimeoutException(reason) - - success = test_release_for_success( - self.tiller, release_name, timeout=timeout, cleanup=cleanup) - if success: - LOG.info("Test passed for release: %s", release_name) - else: - LOG.info("Test failed for release: %s", release_name) - raise tiller_exceptions.TestFailedException(release_name) - def _chart_cleanup(self, prefix, charts, msg): LOG.info('Processing chart cleanup to remove unspecified releases.') @@ -625,6 +338,3 @@ class Armada(object): release) self.tiller.uninstall_release(release) msg['purge'].append(release) - - def get_diff(self, old_chart, old_values, new_chart, values): - return ReleaseDiff(old_chart, old_values, new_chart, values).get_diff() diff --git a/armada/handlers/chart_deploy.py b/armada/handlers/chart_deploy.py new file mode 100644 index 00000000..2ef7622b --- /dev/null +++ b/armada/handlers/chart_deploy.py @@ -0,0 +1,290 @@ +# Copyright 2018 The Armada Authors. +# +# 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 oslo_log import log as logging +import time +import yaml + +from armada import const +from armada.exceptions import armada_exceptions +from armada.handlers.chartbuilder import ChartBuilder +from armada.handlers.test import test_release_for_success +from armada.handlers.release_diff import ReleaseDiff +from armada.handlers.wait import get_wait_for +from armada.exceptions import tiller_exceptions +from armada.utils.release import release_prefixer + +LOG = logging.getLogger(__name__) + + +class ChartDeploy(object): + + def __init__(self, disable_update_pre, disable_update_post, dry_run, + k8s_wait_attempts, k8s_wait_attempt_sleep, timeout, tiller): + self.disable_update_pre = disable_update_pre + self.disable_update_post = disable_update_post + self.dry_run = dry_run + self.k8s_wait_attempts = k8s_wait_attempts + self.k8s_wait_attempt_sleep = k8s_wait_attempt_sleep + self.timeout = timeout + self.tiller = tiller + + def execute(self, chart, cg_test_all_charts, prefix, deployed_releases, + failed_releases): + namespace = chart.get('namespace') + release = chart.get('release') + release_name = release_prefixer(prefix, release) + LOG.info('Processing Chart, release=%s', release_name) + + values = chart.get('values', {}) + pre_actions = {} + post_actions = {} + + result = {} + + protected = chart.get('protected', {}) + p_continue = protected.get('continue_processing', False) + + # Check for existing FAILED release, and purge + if release_name in [rel[0] for rel in failed_releases]: + LOG.info('Purging FAILED release %s before deployment.', + release_name) + if protected: + if p_continue: + LOG.warn( + 'Release %s is `protected`, ' + 'continue_processing=True. Operator must ' + 'handle FAILED release manually.', release_name) + result['protected'] = release_name + return result + else: + LOG.error( + 'Release %s is `protected`, ' + 'continue_processing=False.', release_name) + raise armada_exceptions.ProtectedReleaseException( + release_name) + else: + # Purge the release + self.tiller.uninstall_release(release_name) + result['purge'] = release_name + + wait_values = chart.get('wait', {}) + wait_labels = wait_values.get('labels', {}) + + wait_timeout = self.timeout + if wait_timeout <= 0: + wait_timeout = wait_values.get('timeout', wait_timeout) + + # TODO(MarshM): Deprecated, remove `timeout` key. + deprecated_timeout = chart.get('timeout', None) + if isinstance(deprecated_timeout, int): + LOG.warn('The `timeout` key is deprecated and support ' + 'for this will be removed soon. Use ' + '`wait.timeout` instead.') + if wait_timeout <= 0: + wait_timeout = deprecated_timeout or wait_timeout + + if wait_timeout <= 0: + LOG.info('No Chart timeout specified, using default: %ss', + const.DEFAULT_CHART_TIMEOUT) + wait_timeout = const.DEFAULT_CHART_TIMEOUT + + native_wait = wait_values.get('native', {}) + native_wait_enabled = native_wait.get('enabled', True) + + chartbuilder = ChartBuilder(chart) + new_chart = chartbuilder.get_helm_chart() + + # Begin Chart timeout deadline + deadline = time.time() + wait_timeout + + # TODO(mark-burnett): It may be more robust to directly call + # tiller status to decide whether to install/upgrade rather + # than checking for list membership. + if release_name in [rel[0] for rel in deployed_releases]: + + # indicate to the end user what path we are taking + LOG.info("Upgrading release %s in namespace %s", release_name, + namespace) + # extract the installed chart and installed values from the + # latest release so we can compare to the intended state + old_chart, old_values_string = self.find_release_chart( + deployed_releases, release_name) + + upgrade = chart.get('upgrade', {}) + disable_hooks = upgrade.get('no_hooks', False) + force = upgrade.get('force', False) + recreate_pods = upgrade.get('recreate_pods', False) + + LOG.info("Checking Pre/Post Actions") + if upgrade: + upgrade_pre = upgrade.get('pre', {}) + upgrade_post = upgrade.get('post', {}) + + if not self.disable_update_pre and upgrade_pre: + pre_actions = upgrade_pre + + if not self.disable_update_post and upgrade_post: + post_actions = upgrade_post + + try: + old_values = yaml.safe_load(old_values_string) + except yaml.YAMLError: + chart_desc = '{} (previously deployed)'.format( + old_chart.metadata.name) + raise armada_exceptions.\ + InvalidOverrideValuesYamlException(chart_desc) + + LOG.info('Checking for updates to chart release inputs.') + diff = self.get_diff(old_chart, old_values, new_chart, values) + + if not diff: + LOG.info("Found no updates to chart release inputs") + return result + + LOG.info("Found updates to chart release inputs") + LOG.debug("%s", diff) + result['diff'] = {chart['release']: str(diff)} + + # TODO(MarshM): Add tiller dry-run before upgrade and + # consider deadline impacts + + # do actual update + timer = int(round(deadline - time.time())) + LOG.info('Beginning Upgrade, wait=%s, timeout=%ss', + native_wait_enabled, timer) + tiller_result = self.tiller.update_release( + new_chart, + release_name, + namespace, + pre_actions=pre_actions, + post_actions=post_actions, + disable_hooks=disable_hooks, + values=yaml.safe_dump(values), + wait=native_wait_enabled, + timeout=timer, + force=force, + recreate_pods=recreate_pods) + + LOG.info('Upgrade completed with results from Tiller: %s', + tiller_result.__dict__) + result['upgrade'] = release_name + + # process install + else: + LOG.info("Installing release %s in namespace %s", release_name, + namespace) + + timer = int(round(deadline - time.time())) + LOG.info('Beginning Install, wait=%s, timeout=%ss', + native_wait_enabled, timer) + tiller_result = self.tiller.install_release( + new_chart, + release_name, + namespace, + values=yaml.safe_dump(values), + wait=native_wait_enabled, + timeout=timer) + + LOG.info('Install completed with results from Tiller: %s', + tiller_result.__dict__) + result['install'] = release_name + + timer = int(round(deadline - time.time())) + self._wait_until_ready(release_name, wait_labels, namespace, timer) + + test_chart_override = chart.get('test') + # Use old default value when not using newer `test` key + test_cleanup = True + if test_chart_override is None: + test_this_chart = cg_test_all_charts + elif isinstance(test_chart_override, bool): + LOG.warn('Boolean value for chart `test` key is' + ' deprecated and support for this will' + ' be removed. Use `test.enabled` ' + 'instead.') + test_this_chart = test_chart_override + else: + # NOTE: helm tests are enabled by default + test_this_chart = test_chart_override.get('enabled', True) + test_cleanup = test_chart_override.get('options', {}).get( + 'cleanup', False) + if test_this_chart: + timer = int(round(deadline - time.time())) + self._test_chart(release_name, timer, test_cleanup) + + return result + + def _wait_until_ready(self, release_name, wait_labels, namespace, timeout): + if self.dry_run: + LOG.info( + 'Skipping wait during `dry-run`, would have waited on ' + 'namespace=%s, labels=(%s) for %ss.', namespace, wait_labels, + timeout) + return + + LOG.info('Waiting for release=%s', release_name) + + waits = [ + get_wait_for('job', self.tiller.k8s, skip_if_none_found=True), + get_wait_for('pod', self.tiller.k8s) + ] + deadline = time.time() + timeout + deadline_remaining = timeout + for wait in waits: + if deadline_remaining <= 0: + reason = ( + 'Timeout expired waiting for release=%s' % release_name) + LOG.error(reason) + raise armada_exceptions.ArmadaTimeoutException(reason) + + wait.wait( + labels=wait_labels, + namespace=namespace, + k8s_wait_attempts=self.k8s_wait_attempts, + k8s_wait_attempt_sleep=self.k8s_wait_attempt_sleep, + timeout=timeout) + deadline_remaining = int(round(deadline - time.time())) + + def _test_chart(self, release_name, timeout, cleanup): + if self.dry_run: + LOG.info( + 'Skipping test during `dry-run`, would have tested ' + 'release=%s with timeout %ss.', release_name, timeout) + return True + + if timeout <= 0: + reason = ('Timeout expired before testing ' + 'release %s' % release_name) + LOG.error(reason) + raise armada_exceptions.ArmadaTimeoutException(reason) + + success = test_release_for_success( + self.tiller, release_name, timeout=timeout, cleanup=cleanup) + if success: + LOG.info("Test passed for release: %s", release_name) + else: + LOG.info("Test failed for release: %s", release_name) + raise tiller_exceptions.TestFailedException(release_name) + + def get_diff(self, old_chart, old_values, new_chart, values): + return ReleaseDiff(old_chart, old_values, new_chart, values).get_diff() + + def find_release_chart(self, known_releases, release_name): + ''' + Find a release given a list of known_releases and a release name + ''' + for release, _, chart, values, _ in known_releases: + if release == release_name: + return chart, values diff --git a/armada/schemas/armada-chart-schema.yaml b/armada/schemas/armada-chart-schema.yaml index 954f2032..c555bdde 100644 --- a/armada/schemas/armada-chart-schema.yaml +++ b/armada/schemas/armada-chart-schema.yaml @@ -83,6 +83,14 @@ data: type: integer labels: $ref: "#/definitions/labels" + # Config for helm's native `--wait` param. + native: + type: object + properties: + # TODO: Add separate timeout for native wait? + enabled: + type: boolean + additionalProperties: false additionalProperties: false source: type: object diff --git a/armada/tests/test_utils.py b/armada/tests/test_utils.py index be370d29..65027453 100644 --- a/armada/tests/test_utils.py +++ b/armada/tests/test_utils.py @@ -16,11 +16,34 @@ # License for the specific language governing permissions and limitations # under the License. +import mock import random import string +import testtools +import threading import uuid -import testtools +_mock_thread_safe = False +_mock_call_lock = threading.RLock() + + +# TODO(seaneagan): Get this working. +def makeMockThreadSafe(): + ''' + This attempts to make a subset of the mock library thread safe using + locking, so that the mock call records are accurate. + ''' + global _mock_thread_safe + if not _mock_thread_safe: + unsafe_mock_call = mock.CallableMixin._mock_call + + def safe_mock_call(*args, **kwargs): + with _mock_call_lock: + return unsafe_mock_call(*args, **kwargs) + + mock.CallableMixin._mock_call = safe_mock_call + + _mock_thread_safe = True def rand_uuid_hex(): diff --git a/armada/tests/unit/handlers/test_armada.py b/armada/tests/unit/handlers/test_armada.py index e7ff0e5c..4d593bf7 100644 --- a/armada/tests/unit/handlers/test_armada.py +++ b/armada/tests/unit/handlers/test_armada.py @@ -17,14 +17,17 @@ import yaml from armada import const from armada.handlers import armada +from armada.handlers import chart_deploy from armada.tests.unit import base -from armada.tests.test_utils import AttrDict +from armada.tests.test_utils import AttrDict, makeMockThreadSafe from armada.utils.release import release_prefixer from armada.exceptions import ManifestException from armada.exceptions.override_exceptions import InvalidOverrideValueException from armada.exceptions.validate_exceptions import InvalidManifestException from armada.exceptions import tiller_exceptions -from armada.exceptions.armada_exceptions import ProtectedReleaseException +from armada.exceptions.armada_exceptions import ChartDeployException + +makeMockThreadSafe() TEST_YAML = """ --- @@ -43,7 +46,7 @@ metadata: name: example-group data: description: this is a test - sequenced: False + sequenced: True chart_group: - example-chart-1 - example-chart-2 @@ -136,6 +139,8 @@ data: dependencies: [] wait: timeout: 10 + native: + enabled: false test: enabled: true """ @@ -171,7 +176,10 @@ class ArmadaHandlerTestCase(base.ArmadaTestCase): 'source_dir': CHART_SOURCES[0], 'values': {}, 'wait': { - 'timeout': 10 + 'timeout': 10, + 'native': { + 'enabled': False + } }, 'test': { 'enabled': True @@ -260,7 +268,7 @@ class ArmadaHandlerTestCase(base.ArmadaTestCase): 'name': 'example-group', 'sequenced': - False + True }] } } # yapf: disable @@ -317,6 +325,11 @@ class ArmadaHandlerTestCase(base.ArmadaTestCase): mock_source.source_cleanup.assert_called_with( CHART_SOURCES[counter][0]) + # TODO(seaneagan): Separate ChartDeploy tests into separate module. + # TODO(seaneagan): Once able to make mock library sufficiently thread safe, + # run sync tests for unsequenced as well by moving them to separate test + # class with two separate subclasses which set chart group `sequenced` + # field, one to true, one to false. def _test_sync(self, known_releases, test_success=True, @@ -326,15 +339,15 @@ class ArmadaHandlerTestCase(base.ArmadaTestCase): @mock.patch.object(armada.Armada, 'post_flight_ops') @mock.patch.object(armada.Armada, 'pre_flight_ops') - @mock.patch('armada.handlers.armada.ChartBuilder') + @mock.patch('armada.handlers.chart_deploy.ChartBuilder') @mock.patch('armada.handlers.armada.Tiller') - @mock.patch.object(armada, 'test_release_for_success') + @mock.patch.object(chart_deploy, 'test_release_for_success') def _do_test(mock_test_release_for_success, mock_tiller, mock_chartbuilder, mock_pre_flight, mock_post_flight): # Instantiate Armada object. yaml_documents = list(yaml.safe_load_all(TEST_YAML)) armada_obj = armada.Armada(yaml_documents) - armada_obj.get_diff = mock.Mock() + armada_obj.chart_deploy.get_diff = mock.Mock() chart_group = armada_obj.manifest['armada']['chart_groups'][0] charts = chart_group['chart_group'] @@ -360,7 +373,7 @@ class ArmadaHandlerTestCase(base.ArmadaTestCase): mock_chartbuilder.get_helm_chart.return_value = None # Simulate chart diff, upgrade should only happen if non-empty. - armada_obj.get_diff.return_value = diff + armada_obj.chart_deploy.get_diff.return_value = diff armada_obj.sync() @@ -376,7 +389,8 @@ class ArmadaHandlerTestCase(base.ArmadaTestCase): release_name = release_prefixer(prefix, release) # Simplified check because the actual code uses logical-or's # multiple conditions, so this is enough. - this_chart_should_wait = chart['wait']['timeout'] > 0 + native_wait_enabled = (chart['wait'].get('native', {}).get( + 'enabled', True)) expected_apply = True if release_name not in [x[0] for x in known_releases]: @@ -388,8 +402,8 @@ class ArmadaHandlerTestCase(base.ArmadaTestCase): ['release_prefix'], chart['release']), chart['namespace'], values=yaml.safe_dump(chart['values']), - wait=this_chart_should_wait, - timeout=chart['wait']['timeout'])) + wait=native_wait_enabled, + timeout=mock.ANY)) else: target_release = None for known_release in known_releases: @@ -412,15 +426,16 @@ class ArmadaHandlerTestCase(base.ArmadaTestCase): chart['release']), chart['namespace'], values=yaml.safe_dump(chart['values']), - wait=this_chart_should_wait, - timeout=chart['wait']['timeout'])) + wait=native_wait_enabled, + timeout=mock.ANY)) else: p_continue = protected.get( 'continue_processing', False) if p_continue: continue else: - break + if chart_group['sequenced']: + break if status == const.STATUS_DEPLOYED: if not diff: @@ -446,8 +461,8 @@ class ArmadaHandlerTestCase(base.ArmadaTestCase): force=force, recreate_pods=recreate_pods, values=yaml.safe_dump(chart['values']), - wait=this_chart_should_wait, - timeout=chart['wait']['timeout'])) + wait=native_wait_enabled, + timeout=mock.ANY)) test_chart_override = chart.get('test') # Use old default value when not using newer `test` key @@ -469,6 +484,7 @@ class ArmadaHandlerTestCase(base.ArmadaTestCase): timeout=mock.ANY, cleanup=test_cleanup)) + any_order = not chart_group['sequenced'] # Verify that at least 1 release is either installed or updated. self.assertTrue( len(expected_install_release_calls) >= 1 or @@ -479,28 +495,28 @@ class ArmadaHandlerTestCase(base.ArmadaTestCase): len(expected_install_release_calls), m_tiller.install_release.call_count) m_tiller.install_release.assert_has_calls( - expected_install_release_calls) + expected_install_release_calls, any_order=any_order) # Verify that the expected number of deployed releases are # updated with expected arguments. self.assertEqual( len(expected_update_release_calls), m_tiller.update_release.call_count) m_tiller.update_release.assert_has_calls( - expected_update_release_calls) + expected_update_release_calls, any_order=any_order) # Verify that the expected number of deployed releases are # uninstalled with expected arguments. self.assertEqual( len(expected_uninstall_release_calls), m_tiller.uninstall_release.call_count) m_tiller.uninstall_release.assert_has_calls( - expected_uninstall_release_calls) + expected_uninstall_release_calls, any_order=any_order) # Verify that the expected number of deployed releases are # tested with expected arguments. self.assertEqual( len(expected_test_release_for_success_calls), mock_test_release_for_success.call_count) mock_test_release_for_success.assert_has_calls( - expected_test_release_for_success_calls) + expected_test_release_for_success_calls, any_order=any_order) _do_test() @@ -566,66 +582,21 @@ class ArmadaHandlerTestCase(base.ArmadaTestCase): def _test_method(): self._test_sync(known_releases) - self.assertRaises(ProtectedReleaseException, _test_method) + self.assertRaises(ChartDeployException, _test_method) def test_armada_sync_test_failure(self): def _test_method(): self._test_sync([], test_success=False) - self.assertRaises(tiller_exceptions.TestFailedException, _test_method) + self.assertRaises(ChartDeployException, _test_method) def test_armada_sync_test_failure_to_run(self): def _test_method(): self._test_sync([], test_failure_to_run=True) - self.assertRaises(tiller_exceptions.ReleaseException, _test_method) - - @mock.patch.object(armada.Armada, 'post_flight_ops') - @mock.patch.object(armada.Armada, 'pre_flight_ops') - @mock.patch('armada.handlers.armada.ChartBuilder') - @mock.patch('armada.handlers.armada.Tiller') - def test_install(self, mock_tiller, mock_chartbuilder, mock_pre_flight, - mock_post_flight): - '''Test install functionality from the sync() method''' - - # Instantiate Armada object. - yaml_documents = list(yaml.safe_load_all(TEST_YAML)) - armada_obj = armada.Armada(yaml_documents) - - charts = armada_obj.manifest['armada']['chart_groups'][0][ - 'chart_group'] - chart_1 = charts[0]['chart'] - chart_2 = charts[1]['chart'] - - # Mock irrelevant methods called by `armada.sync()`. - mock_tiller.list_charts.return_value = [] - mock_chartbuilder.get_source_path.return_value = None - mock_chartbuilder.get_helm_chart.return_value = None - - armada_obj.sync() - - # Check params that should be passed to `tiller.install_release()`. - method_calls = [ - mock.call( - mock_chartbuilder().get_helm_chart(), - "{}-{}".format(armada_obj.manifest['armada']['release_prefix'], - chart_1['release']), - chart_1['namespace'], - values=yaml.safe_dump(chart_1['values']), - timeout=10, - wait=True), - mock.call( - mock_chartbuilder().get_helm_chart(), - "{}-{}".format(armada_obj.manifest['armada']['release_prefix'], - chart_2['release']), - chart_2['namespace'], - values=yaml.safe_dump(chart_2['values']), - timeout=10, - wait=True) - ] - mock_tiller.return_value.install_release.assert_has_calls(method_calls) + self.assertRaises(ChartDeployException, _test_method) class ArmadaNegativeHandlerTestCase(base.ArmadaTestCase): diff --git a/doc/source/operations/guide-build-armada-yaml.rst b/doc/source/operations/guide-build-armada-yaml.rst index caabeb03..e0fad89e 100644 --- a/doc/source/operations/guide-build-armada-yaml.rst +++ b/doc/source/operations/guide-build-armada-yaml.rst @@ -99,12 +99,12 @@ Chart +-----------------+----------+---------------------------------------------------------------------------------------+ | namespace | string | namespace of your chart | +-----------------+----------+---------------------------------------------------------------------------------------+ -| wait | object | contains wait information such as (timeout, labels) | +| wait | object | See Wait_. | +-----------------+----------+---------------------------------------------------------------------------------------+ | protected | object | do not delete FAILED releases when encountered from previous run (provide the | | | | 'continue_processing' bool to continue or halt execution (default: halt)) | +-----------------+----------+---------------------------------------------------------------------------------------+ -| test | object | Run helm tests on the chart after install/upgrade (default enabled) | +| test | object | See Test_. | +-----------------+----------+---------------------------------------------------------------------------------------+ | install | object | install the chart into your Kubernetes cluster | +-----------------+----------+---------------------------------------------------------------------------------------+ @@ -119,15 +119,41 @@ Chart | timeout | int | time (in seconds) allotted for chart to deploy when 'wait' flag is set (DEPRECATED) | +-----------------+----------+---------------------------------------------------------------------------------------+ +Wait +^^^^ + ++-------------+----------+--------------------------------------------------------------------+ +| keyword | type | action | ++=============+==========+====================================================================+ +| native | object | See `Wait Native`_. | ++-------------+----------+--------------------------------------------------------------------+ +| timeout | int | time (in seconds) to wait for chart to deploy | ++-------------+----------+--------------------------------------------------------------------+ +| labels | object | k:v mapping of labels to select Kubernetes resources | ++-------------+----------+--------------------------------------------------------------------+ + +Wait Native +^^^^^^^^^^^ + +Config for the native ``helm (install|upgrade) --wait`` flag. + ++-------------+----------+--------------------------------------------------------------------+ +| keyword | type | action | ++=============+==========+====================================================================+ +| enabled | boolean | defaults to true | ++-------------+----------+--------------------------------------------------------------------+ + Test ^^^^ +Run helm tests on the chart after install/upgrade. + +-------------+----------+--------------------------------------------------------------------+ | keyword | type | action | +=============+==========+====================================================================+ | enabled | bool | whether to enable/disable helm tests for this chart (default True) | +-------------+----------+--------------------------------------------------------------------+ -| options | object | options to pass through to helm | +| options | object | See `Test Options`_. | +-------------+----------+--------------------------------------------------------------------+ .. note:: @@ -142,8 +168,13 @@ Test deprecated and will be removed. The ``cleanup`` option below is set to true in this case for backward compatibility. -Test - Options -^^^^^^^^^^^^^^ +.. _test_options: + + +Test Options +^^^^^^^^^^^^ + +Test options to pass through directly to helm. +-------------+----------+---------------------------------------------------------------+ | keyword | type | action |