Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 92CB5200B38 for ; Fri, 8 Jul 2016 22:33:48 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 914B2160A5A; Fri, 8 Jul 2016 20:33:48 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 40A7B160A36 for ; Fri, 8 Jul 2016 22:33:47 +0200 (CEST) Received: (qmail 76366 invoked by uid 500); 8 Jul 2016 20:33:46 -0000 Mailing-List: contact commits-help@aurora.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@aurora.apache.org Delivered-To: mailing list commits@aurora.apache.org Received: (qmail 76356 invoked by uid 99); 8 Jul 2016 20:33:46 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 08 Jul 2016 20:33:46 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 0531FDFFF8; Fri, 8 Jul 2016 20:33:45 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: jcohen@apache.org To: commits@aurora.apache.org Message-Id: <0f5d816007974e38ac8fd8674edc2d97@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: aurora git commit: AURORA-1710 Make 'tier' required and remove support for 'production' flag in Job configuration - CLI changes Date: Fri, 8 Jul 2016 20:33:45 +0000 (UTC) archived-at: Fri, 08 Jul 2016 20:33:48 -0000 Repository: aurora Updated Branches: refs/heads/master 70d0ef266 -> 7701d218c AURORA-1710 Make 'tier' required and remove support for 'production' flag in Job configuration - CLI changes Bugs closed: AURORA-1710 Reviewed at https://reviews.apache.org/r/49048/ Project: http://git-wip-us.apache.org/repos/asf/aurora/repo Commit: http://git-wip-us.apache.org/repos/asf/aurora/commit/7701d218 Tree: http://git-wip-us.apache.org/repos/asf/aurora/tree/7701d218 Diff: http://git-wip-us.apache.org/repos/asf/aurora/diff/7701d218 Branch: refs/heads/master Commit: 7701d218cd9c22cb4a2f107d28695d57e679b402 Parents: 70d0ef2 Author: Mehrdad Nurolahzade Authored: Fri Jul 8 15:33:24 2016 -0500 Committer: Joshua Cohen Committed: Fri Jul 8 15:33:24 2016 -0500 ---------------------------------------------------------------------- RELEASE-NOTES.md | 14 ++++ .../python/apache/aurora/client/api/__init__.py | 4 + .../python/apache/aurora/client/cli/context.py | 64 ++++++++++++---- src/main/python/apache/aurora/client/config.py | 25 +++++++ .../aurora/client/cli/test_command_hooks.py | 1 + .../apache/aurora/client/cli/test_context.py | 77 +++++++++++++++++++- .../apache/aurora/client/cli/test_create.py | 12 ++- .../apache/aurora/client/cli/test_cron.py | 5 ++ .../apache/aurora/client/cli/test_plugins.py | 1 + .../apache/aurora/client/cli/test_restart.py | 6 ++ .../python/apache/aurora/client/cli/util.py | 28 ++++++- .../python/apache/aurora/client/test_config.py | 20 +++++ 12 files changed, 240 insertions(+), 17 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/aurora/blob/7701d218/RELEASE-NOTES.md ---------------------------------------------------------------------- diff --git a/RELEASE-NOTES.md b/RELEASE-NOTES.md index 29d224d..3d1e7bc 100644 --- a/RELEASE-NOTES.md +++ b/RELEASE-NOTES.md @@ -1,3 +1,17 @@ +0.16.0 (Not yet released) +====== + +### New/Updated: + +- Scheduler command line behavior has been modified to warn users of the deprecation of `production` + attribute in `Job` thrift struct. The scheduler is queried for tier configurations and the user's + choice of `tier` and `production` attributes is revised, if necessary. If `tier` is already set, + the `production` attribute might be adjusted to match the `tier` selection. Otherwise, `tier` is + selected based on the value of `production` attribute. If a matching tier is not found, the + `default` tier from tier configuration file (`tiers.json`) is used. + +### Deprecations and removals: + 0.15.0 ====== http://git-wip-us.apache.org/repos/asf/aurora/blob/7701d218/src/main/python/apache/aurora/client/api/__init__.py ---------------------------------------------------------------------- diff --git a/src/main/python/apache/aurora/client/api/__init__.py b/src/main/python/apache/aurora/client/api/__init__.py index 68baf8f..ec2c786 100644 --- a/src/main/python/apache/aurora/client/api/__init__.py +++ b/src/main/python/apache/aurora/client/api/__init__.py @@ -294,6 +294,10 @@ class AuroraClientAPI(object): Resource(ramMb=ram), Resource(diskMb=disk)]))) + def get_tier_configs(self): + log.debug("Getting tier configurations") + return self._scheduler_proxy.getTierConfigs() + def force_task_state(self, task_id, status): log.info("Requesting that task %s transition to state %s" % (task_id, status)) return self._scheduler_proxy.forceTaskState(task_id, status) http://git-wip-us.apache.org/repos/asf/aurora/blob/7701d218/src/main/python/apache/aurora/client/cli/context.py ---------------------------------------------------------------------- diff --git a/src/main/python/apache/aurora/client/cli/context.py b/src/main/python/apache/aurora/client/cli/context.py index 9b15118..d7fbf80 100644 --- a/src/main/python/apache/aurora/client/cli/context.py +++ b/src/main/python/apache/aurora/client/cli/context.py @@ -18,6 +18,8 @@ import functools import logging from fnmatch import fnmatch +from pystachio import Boolean, String + from apache.aurora.client.api import AuroraClientAPI, SchedulerProxy from apache.aurora.client.base import AURORA_V2_USER_AGENT_NAME, combine_messages from apache.aurora.client.cli import ( @@ -28,7 +30,7 @@ from apache.aurora.client.cli import ( EXIT_INVALID_PARAMETER, Context ) -from apache.aurora.client.config import get_config +from apache.aurora.client.config import AnnotatedAuroraConfig, get_config from apache.aurora.client.hooks.hooked_api import HookedAuroraClientAPI from apache.aurora.common.aurora_job_key import AuroraJobKey from apache.aurora.common.clusters import CLUSTERS @@ -99,6 +101,52 @@ class AuroraCommandContext(Context): apis[cluster] = api return add_auth_error_handler(apis[cluster]) + def log_response_and_raise(self, resp, err_code=EXIT_API_ERROR, err_msg="Command failure:"): + if resp.responseCode == ResponseCode.OK: + msg = combine_messages(resp) + if msg: + logging.info(msg) + else: + self.print_err(err_msg) + self.print_err("\t%s" % combine_messages(resp)) + if resp.responseCode == ResponseCode.LOCK_ERROR: + self.print_err("\t%s" % self.LOCK_ERROR_MSG) + raise self.CommandErrorLogged(err_code, err_msg) + + def _get_tier_configurations(self, cluster): + api = self.get_api(cluster) + resp = api.get_tier_configs() + self.log_response_and_raise(resp, err_msg="Error getting tier configurations.") + return resp.result.getTierConfigResult + + def _get_config_with_production_and_tier(self, cluster, config): + tier_configurations = self._get_tier_configurations(cluster) + + def to_bool(value): + return Boolean.coerce(value) + + def production_tier_filter(tier): + return not to_bool(tier.settings['preemptible']) and not to_bool(tier.settings['revocable']) + + task = config.job().taskConfig + if task.tier is None: + backfill_args = { + 'tier': String( + next( + (t.name for t in tier_configurations.tiers if + production_tier_filter(t) == task.production), + tier_configurations.defaultTierName)) + } + else: + backfill_args = { + 'production': Boolean( + next( + (not to_bool(t.settings['preemptible']) for t in tier_configurations.tiers if + t.name == task.tier), + task.production)) + } + return AnnotatedAuroraConfig(config.raw()(**backfill_args)) + def get_job_config_optional(self, jobkey, config_file): """Loads a job configuration if provided.""" return self.get_job_config(jobkey, config_file) if config_file is not None else None @@ -122,22 +170,10 @@ class AuroraCommandContext(Context): check_result = result.raw().check() if not check_result.ok(): raise self.CommandError(EXIT_INVALID_CONFIGURATION, check_result) - return result + return self._get_config_with_production_and_tier(jobkey.cluster, result) except Exception as e: raise self.CommandError(EXIT_INVALID_CONFIGURATION, 'Error loading configuration: %s' % e) - def log_response_and_raise(self, resp, err_code=EXIT_API_ERROR, err_msg="Command failure:"): - if resp.responseCode == ResponseCode.OK: - msg = combine_messages(resp) - if msg: - logging.info(msg) - else: - self.print_err(err_msg) - self.print_err("\t%s" % combine_messages(resp)) - if resp.responseCode == ResponseCode.LOCK_ERROR: - self.print_err("\t%s" % self.LOCK_ERROR_MSG) - raise self.CommandErrorLogged(err_code, err_msg) - def get_job_list(self, clusters, role=None): """Get a list of jobs from a group of clusters. :param clusters: the clusters to query for jobs http://git-wip-us.apache.org/repos/asf/aurora/blob/7701d218/src/main/python/apache/aurora/client/config.py ---------------------------------------------------------------------- diff --git a/src/main/python/apache/aurora/client/config.py b/src/main/python/apache/aurora/client/config.py index 2fc1255..96cd9dd 100644 --- a/src/main/python/apache/aurora/client/config.py +++ b/src/main/python/apache/aurora/client/config.py @@ -22,6 +22,8 @@ import math import re import sys +from twitter.common import log + from apache.aurora.client import binding_helper from apache.aurora.client.base import die from apache.aurora.config import AuroraConfig @@ -114,10 +116,33 @@ def _validate_update_config(config): (watch_secs, target_watch, initial_interval_secs, max_consecutive_failures, interval_secs)) +PRODUCTION_DEPRECATED_WARNING = ( + 'Job configuration attribute \'production\' is deprecated.\n' + 'Use \'tier\' attribute instead. For more information please refer to \n' + 'http://aurora.apache.org/documentation/latest/reference/configuration/#job-objects') + + +def deprecation_warning(text): + log.warning('') + log.warning('*' * 80) + log.warning('* The command you ran is deprecated and will soon break!') + for line in text.split('\n'): + log.warning('* %s' % line) + log.warning('*' * 80) + log.warning('') + + +def _validate_deprecated_config(config): + task = config.job().taskConfig + if task.production and task.tier is None: + deprecation_warning(PRODUCTION_DEPRECATED_WARNING) + + def validate_config(config, env=None): _validate_update_config(config) _validate_announce_configuration(config) _validate_environment_name(config) + _validate_deprecated_config(config) class GlobalHookRegistry(object): http://git-wip-us.apache.org/repos/asf/aurora/blob/7701d218/src/test/python/apache/aurora/client/cli/test_command_hooks.py ---------------------------------------------------------------------- diff --git a/src/test/python/apache/aurora/client/cli/test_command_hooks.py b/src/test/python/apache/aurora/client/cli/test_command_hooks.py index 2130f1f..a44a25f 100644 --- a/src/test/python/apache/aurora/client/cli/test_command_hooks.py +++ b/src/test/python/apache/aurora/client/cli/test_command_hooks.py @@ -113,6 +113,7 @@ class TestClientCreateCommand(AuroraClientCommandTest): lambda x: self.create_mock_status_query_result(ScheduleStatus.RUNNING)) api = mock_context.get_api("west") api.create_job.return_value = self.get_createjob_response() + api.get_tier_configs.return_value = self.get_mock_tier_configurations() with temporary_file() as fp: fp.write(self.get_valid_config()) http://git-wip-us.apache.org/repos/asf/aurora/blob/7701d218/src/test/python/apache/aurora/client/cli/test_context.py ---------------------------------------------------------------------- diff --git a/src/test/python/apache/aurora/client/cli/test_context.py b/src/test/python/apache/aurora/client/cli/test_context.py index 204ca09..50e40e4 100644 --- a/src/test/python/apache/aurora/client/cli/test_context.py +++ b/src/test/python/apache/aurora/client/cli/test_context.py @@ -13,9 +13,10 @@ # import mock import pytest +from twitter.common.contextutil import temporary_file from apache.aurora.client.api import AuroraClientAPI, SchedulerProxy -from apache.aurora.client.cli import EXIT_AUTH_ERROR, Context +from apache.aurora.client.cli import EXIT_AUTH_ERROR, CommandOption, Context from apache.aurora.client.cli.context import AuroraCommandContext from apache.aurora.client.hooks.hooked_api import HookedAuroraClientAPI from apache.aurora.common.aurora_job_key import AuroraJobKey @@ -23,6 +24,7 @@ from apache.aurora.common.cluster import Cluster from apache.aurora.common.clusters import CLUSTERS from ...api_util import SchedulerProxyApiSpec +from .util import AuroraClientCommandTest, FakeAuroraCommandContext TEST_CLUSTER = Cluster(name='some-cluster', auth_mechanism='nothing', scheduler_uri='nowhere') @@ -75,3 +77,76 @@ def test_handles_api_auth_error(): assert e.value.code == EXIT_AUTH_ERROR assert mock_scheduler_proxy.killTasks.call_count == 1 + + +def create_mock_options(): + mock_options = mock.Mock(spec=CommandOption) + mock_options.read_json = False + mock_options.bindings = () + return mock_options + + +def create_test_config(extra=''): + return AuroraClientCommandTest.get_test_config( + AuroraClientCommandTest.CONFIG_BASE, + AuroraClientCommandTest.TEST_CLUSTER, + AuroraClientCommandTest.TEST_ROLE, + AuroraClientCommandTest.TEST_ENV, + AuroraClientCommandTest.TEST_JOB, + extra) + + +def test_get_config_with_production_and_tier_is_preemptible(): + context = FakeAuroraCommandContext() + context.set_options(create_mock_options()) + with CLUSTERS.patch(AuroraClientCommandTest.TEST_CLUSTERS.values()): + api = context.get_api(TEST_CLUSTER.name) + api.get_tier_configs.return_value = AuroraClientCommandTest.get_mock_tier_configurations() + with temporary_file() as fp: + fp.write(create_test_config()) + fp.flush() + config = context.get_job_config(AuroraClientCommandTest.TEST_JOBKEY, fp.name) + assert not config.job().taskConfig.production + assert config.job().taskConfig.tier == AuroraClientCommandTest.PREEMPTIBLE_TIER.name + + +def test_get_config_with_production_and_tier_is_preferred(): + context = FakeAuroraCommandContext() + context.set_options(create_mock_options()) + with CLUSTERS.patch(AuroraClientCommandTest.TEST_CLUSTERS.values()): + api = context.get_api(TEST_CLUSTER.name) + api.get_tier_configs.return_value = AuroraClientCommandTest.get_mock_tier_configurations() + with temporary_file() as fp: + fp.write(create_test_config("production='true',")) + fp.flush() + config = context.get_job_config(AuroraClientCommandTest.TEST_JOBKEY, fp.name) + assert config.job().taskConfig.production + assert config.job().taskConfig.tier == AuroraClientCommandTest.PREFERRED_TIER.name + + +def test_get_config_with_production_and_tier_resets_production(): + context = FakeAuroraCommandContext() + context.set_options(create_mock_options()) + with CLUSTERS.patch(AuroraClientCommandTest.TEST_CLUSTERS.values()): + api = context.get_api(TEST_CLUSTER.name) + api.get_tier_configs.return_value = AuroraClientCommandTest.get_mock_tier_configurations() + with temporary_file() as fp: + fp.write(create_test_config("production='true', tier='revocable',")) + fp.flush() + config = context.get_job_config(AuroraClientCommandTest.TEST_JOBKEY, fp.name) + assert not config.job().taskConfig.production + assert config.job().taskConfig.tier == AuroraClientCommandTest.REVOCABLE_TIER.name + + +def test_get_config_with_production_and_tier_sets_production(): + context = FakeAuroraCommandContext() + context.set_options(create_mock_options()) + with CLUSTERS.patch(AuroraClientCommandTest.TEST_CLUSTERS.values()): + api = context.get_api(TEST_CLUSTER.name) + api.get_tier_configs.return_value = AuroraClientCommandTest.get_mock_tier_configurations() + with temporary_file() as fp: + fp.write(create_test_config("tier='preferred',")) + fp.flush() + config = context.get_job_config(AuroraClientCommandTest.TEST_JOBKEY, fp.name) + assert config.job().taskConfig.production + assert config.job().taskConfig.tier == AuroraClientCommandTest.PREFERRED_TIER.name http://git-wip-us.apache.org/repos/asf/aurora/blob/7701d218/src/test/python/apache/aurora/client/cli/test_create.py ---------------------------------------------------------------------- diff --git a/src/test/python/apache/aurora/client/cli/test_create.py b/src/test/python/apache/aurora/client/cli/test_create.py index 8c27e2b..3b09bb2 100644 --- a/src/test/python/apache/aurora/client/cli/test_create.py +++ b/src/test/python/apache/aurora/client/cli/test_create.py @@ -135,6 +135,7 @@ class TestClientCreateCommand(AuroraClientCommandTest): self.create_mock_status_query_result(ScheduleStatus.RUNNING)) api = mock_context.get_api('west') api.create_job.return_value = self.get_createjob_response() + api.get_tier_configs.return_value = self.get_mock_tier_configurations() # This is the real test: invoke create as if it had been called by the command line. with temporary_file() as fp: @@ -163,6 +164,7 @@ class TestClientCreateCommand(AuroraClientCommandTest): self.create_mock_status_query_result(ScheduleStatus.RUNNING)) api = mock_context.get_api('west') api.create_job.return_value = self.get_createjob_response() + api.get_tier_configs.return_value = self.get_mock_tier_configurations() with temporary_file() as fp: fp.write(self.get_valid_config()) fp.flush() @@ -191,6 +193,7 @@ class TestClientCreateCommand(AuroraClientCommandTest): mock_context.add_expected_status_query_result(self.create_mock_status_query_result(result)) api = mock_context.get_api('west') api.create_job.return_value = self.get_createjob_response() + api.get_tier_configs.return_value = self.get_mock_tier_configurations() with temporary_file() as fp: fp.write(self.get_valid_config()) fp.flush() @@ -210,6 +213,7 @@ class TestClientCreateCommand(AuroraClientCommandTest): self.create_mock_status_query_result(ScheduleStatus.INIT)) api = mock_context.get_api('west') api.create_job.return_value = self.get_failed_createjob_response() + api.get_tier_configs.return_value = self.get_mock_tier_configurations() with temporary_file() as fp: fp.write(self.get_valid_config()) fp.flush() @@ -288,6 +292,7 @@ class TestClientCreateCommand(AuroraClientCommandTest): self.create_mock_status_query_result(ScheduleStatus.RUNNING)) api = mock_context.get_api('west') api.create_job.return_value = self.get_createjob_response() + api.get_tier_configs.return_value = self.get_mock_tier_configurations() with temporary_file() as fp: fp.write(self.get_valid_config()) @@ -318,6 +323,7 @@ class TestClientCreateCommand(AuroraClientCommandTest): api = mock_context.get_api('west') api.create_job.return_value = self.get_createjob_response() + api.get_tier_configs.return_value = self.get_mock_tier_configurations() with temporary_file() as fp: fp.write(self.get_valid_config()) @@ -337,6 +343,7 @@ class TestClientCreateCommand(AuroraClientCommandTest): self.create_mock_status_query_result(ScheduleStatus.INIT)) api = mock_context.get_api('west') api.create_job.return_value = self.get_failed_createjob_response() + api.get_tier_configs.return_value = self.get_mock_tier_configurations() with temporary_file() as fp: fp.write(self.get_valid_config()) fp.flush() @@ -366,6 +373,7 @@ class TestClientCreateCommand(AuroraClientCommandTest): self.create_mock_status_query_result(ScheduleStatus.RUNNING)) api = mock_context.get_api('west') api.create_job.return_value = self.get_createjob_response() + api.get_tier_configs.return_value = self.get_mock_tier_configurations() # This is the real test: invoke create as if it had been called by the command line. with temporary_file() as fp: @@ -404,11 +412,13 @@ class TestClientCreateCommand(AuroraClientCommandTest): fp.name]) assert result == EXIT_INVALID_CONFIGURATION assert mock_context.get_out() == [] - assert "Error loading configuration: TypeCheck(FAILED):" in mock_context.get_err()[0] + assert "Error loading configuration: " in mock_context.get_err()[0] def test_create_cron_job_fails(self): """Test a cron job is not accepted.""" mock_context = FakeAuroraCommandContext() + api = mock_context.get_api('west') + api.get_tier_configs.return_value = self.get_mock_tier_configurations() with patch('apache.aurora.client.cli.jobs.Job.create_context', return_value=mock_context): with temporary_file() as fp: fp.write(self.get_valid_cron_config()) http://git-wip-us.apache.org/repos/asf/aurora/blob/7701d218/src/test/python/apache/aurora/client/cli/test_cron.py ---------------------------------------------------------------------- diff --git a/src/test/python/apache/aurora/client/cli/test_cron.py b/src/test/python/apache/aurora/client/cli/test_cron.py index f3c522e..61690d7 100644 --- a/src/test/python/apache/aurora/client/cli/test_cron.py +++ b/src/test/python/apache/aurora/client/cli/test_cron.py @@ -43,6 +43,7 @@ class TestCronNoun(AuroraClientCommandTest): api = mock_context.get_api('west') api.schedule_cron.return_value = self.create_simple_success_response() + api.get_tier_configs.return_value = self.get_mock_tier_configurations() with temporary_file() as fp: fp.write(self.get_valid_cron_config()) fp.flush() @@ -62,6 +63,7 @@ class TestCronNoun(AuroraClientCommandTest): with patch('apache.aurora.client.cli.cron.CronNoun.create_context', return_value=mock_context): api = mock_context.get_api('west') api.schedule_cron.return_value = self.create_error_response() + api.get_tier_configs.return_value = self.get_mock_tier_configurations() with temporary_file() as fp: fp.write(self.get_valid_cron_config()) fp.flush() @@ -76,6 +78,8 @@ class TestCronNoun(AuroraClientCommandTest): def test_schedule_failed_non_cron(self): mock_context = FakeAuroraCommandContext() + api = mock_context.get_api('west') + api.get_tier_configs.return_value = self.get_mock_tier_configurations() with patch('apache.aurora.client.cli.cron.CronNoun.create_context', return_value=mock_context): with temporary_file() as fp: fp.write(self.get_valid_config()) @@ -104,6 +108,7 @@ class TestCronNoun(AuroraClientCommandTest): with patch('apache.aurora.client.cli.cron.CronNoun.create_context', return_value=mock_context): api = mock_context.get_api("west") api.schedule_cron.return_value = self.create_simple_success_response() + api.get_tier_configs.return_value = self.get_mock_tier_configurations() with temporary_file() as fp: fp.write(self.get_valid_cron_config()) fp.flush() http://git-wip-us.apache.org/repos/asf/aurora/blob/7701d218/src/test/python/apache/aurora/client/cli/test_plugins.py ---------------------------------------------------------------------- diff --git a/src/test/python/apache/aurora/client/cli/test_plugins.py b/src/test/python/apache/aurora/client/cli/test_plugins.py index a545fec..762735e 100644 --- a/src/test/python/apache/aurora/client/cli/test_plugins.py +++ b/src/test/python/apache/aurora/client/cli/test_plugins.py @@ -110,6 +110,7 @@ class TestPlugins(AuroraClientCommandTest): self.create_mock_status_query_result(ScheduleStatus.RUNNING)) api = mock_context.get_api('west') api.create_job.return_value = self.get_createjob_response() + api.get_tier_configs.return_value = self.get_mock_tier_configurations() # This is the real test: invoke create as if it had been called by the command line. with temporary_file() as fp: http://git-wip-us.apache.org/repos/asf/aurora/blob/7701d218/src/test/python/apache/aurora/client/cli/test_restart.py ---------------------------------------------------------------------- diff --git a/src/test/python/apache/aurora/client/cli/test_restart.py b/src/test/python/apache/aurora/client/cli/test_restart.py index 967d560..cb4adc5 100644 --- a/src/test/python/apache/aurora/client/cli/test_restart.py +++ b/src/test/python/apache/aurora/client/cli/test_restart.py @@ -98,6 +98,7 @@ class TestRestartCommand(AuroraClientCommandTest): (mock_api, mock_scheduler_proxy) = self.create_mock_api() mock_health_check = self.setup_health_checks() self.setup_mock_scheduler_for_simple_restart(mock_api) + mock_scheduler_proxy.getTierConfigs.return_value = self.get_mock_tier_configurations() with contextlib.nested( patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy), patch('apache.aurora.client.api.instance_watcher.StatusHealthCheck', @@ -170,6 +171,7 @@ class TestRestartCommand(AuroraClientCommandTest): mock_health_check = self.setup_health_checks() self.setup_mock_scheduler_for_simple_restart(mock_api) mock_scheduler_proxy.getTasksWithoutConfigs.return_value = self.create_error_response() + mock_scheduler_proxy.getTierConfigs.return_value = self.get_mock_tier_configurations() with contextlib.nested( patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy), patch('apache.aurora.client.api.instance_watcher.StatusHealthCheck', @@ -193,6 +195,7 @@ class TestRestartCommand(AuroraClientCommandTest): self.setup_mock_scheduler_for_simple_restart(mock_api) # Make getTasksWithoutConfigs return an error, which is what happens when a job is not found. mock_scheduler_proxy.getTasksWithoutConfigs.return_value = self.create_error_response() + mock_scheduler_proxy.getTierConfigs.return_value = self.get_mock_tier_configurations() with contextlib.nested( patch('apache.aurora.client.cli.context.AuroraCommandContext.print_err', side_effect=mock_io.put), @@ -223,6 +226,7 @@ class TestRestartCommand(AuroraClientCommandTest): mock_health_check = self.setup_health_checks() self.setup_mock_scheduler_for_simple_restart(mock_api) mock_scheduler_proxy.restartShards.return_value = self.create_error_response() + mock_scheduler_proxy.getTierConfigs.return_value = self.get_mock_tier_configurations() with contextlib.nested( patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy), patch('apache.aurora.client.api.instance_watcher.StatusHealthCheck', @@ -265,6 +269,7 @@ class TestRestartCommand(AuroraClientCommandTest): (mock_api, mock_scheduler_proxy) = self.create_mock_api() mock_health_check = self.setup_health_checks() self.setup_mock_scheduler_for_simple_restart(mock_api) + mock_scheduler_proxy.getTierConfigs.return_value = self.get_mock_tier_configurations() with contextlib.nested( patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy), patch('apache.aurora.client.api.instance_watcher.StatusHealthCheck', @@ -291,6 +296,7 @@ class TestRestartCommand(AuroraClientCommandTest): mock_health_check = self.setup_health_checks() self.setup_mock_scheduler_for_simple_restart(mock_api) mock_scheduler_proxy.restartShards.return_value = self.create_error_response() + mock_scheduler_proxy.getTierConfigs.return_value = self.get_mock_tier_configurations() with contextlib.nested( patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy), patch('apache.aurora.client.api.instance_watcher.StatusHealthCheck', http://git-wip-us.apache.org/repos/asf/aurora/blob/7701d218/src/test/python/apache/aurora/client/cli/util.py ---------------------------------------------------------------------- diff --git a/src/test/python/apache/aurora/client/cli/util.py b/src/test/python/apache/aurora/client/cli/util.py index 7b4558e..aac9f9c 100644 --- a/src/test/python/apache/aurora/client/cli/util.py +++ b/src/test/python/apache/aurora/client/cli/util.py @@ -29,6 +29,7 @@ from gen.apache.aurora.api.constants import ACTIVE_STATES from gen.apache.aurora.api.ttypes import ( AssignedTask, ExecutorConfig, + GetTierConfigResult, JobKey, Response, ResponseCode, @@ -39,7 +40,8 @@ from gen.apache.aurora.api.ttypes import ( ScheduleStatusResult, TaskConfig, TaskEvent, - TaskQuery + TaskQuery, + TierConfig ) @@ -386,6 +388,30 @@ jobs = [HELLO_WORLD] def assert_lock_message(cls, context): assert [line for line in context.get_err() if line == "\t%s" % context.LOCK_ERROR_MSG] + PREFERRED_TIER = TierConfig( + name='preferred', + settings={'preemptible': 'false', 'revocable': 'false'} + ) + + PREEMPTIBLE_TIER = TierConfig( + name='preemptible', + settings={'preemptible': 'true', 'revocable': 'false'} + ) + + REVOCABLE_TIER = TierConfig( + name='revocable', + settings={'preemptible': 'true', 'revocable': 'true'} + ) + + @classmethod + def get_mock_tier_configurations(cls): + response = cls.create_simple_success_response() + response.result = Result(getTierConfigResult=GetTierConfigResult( + defaultTierName=cls.PREEMPTIBLE_TIER.name, + tiers=frozenset([cls.PREFERRED_TIER, cls.PREEMPTIBLE_TIER, cls.REVOCABLE_TIER]) + )) + return response + class IOMock(object): def __init__(self): http://git-wip-us.apache.org/repos/asf/aurora/blob/7701d218/src/test/python/apache/aurora/client/test_config.py ---------------------------------------------------------------------- diff --git a/src/test/python/apache/aurora/client/test_config.py b/src/test/python/apache/aurora/client/test_config.py index b1a3c18..4742fa2 100644 --- a/src/test/python/apache/aurora/client/test_config.py +++ b/src/test/python/apache/aurora/client/test_config.py @@ -15,11 +15,13 @@ import os from io import BytesIO +import mock import pytest from twitter.common.contextutil import temporary_dir from apache.aurora.client import config from apache.aurora.client.config import get_config as get_aurora_config +from apache.aurora.client.config import PRODUCTION_DEPRECATED_WARNING from apache.aurora.config import AuroraConfig from apache.aurora.config.loader import AuroraConfigLoader from apache.aurora.config.schema.base import ( @@ -221,3 +223,21 @@ def test_update_config_fails_insufficient_watch_secs_equal_to_target(): with pytest.raises(SystemExit): config._validate_update_config(AuroraConfig(base_job)) + + +def test_validate_deprecated_config_adds_warning_for_production(): + job = Job(name='hello_world', role='john_doe', cluster='test-cluster', environment='test', + task=Task(name='main', processes=[], resources=Resources(cpu=0.1, ram=64 * MB, disk=64 * MB)), + production='true') + with mock.patch('apache.aurora.client.config.deprecation_warning') as mock_warning: + config._validate_deprecated_config(AuroraConfig(job)) + mock_warning.assert_called_once_with(PRODUCTION_DEPRECATED_WARNING) + + +def test_validate_deprecated_config_adds_no_warning_when_tier_is_set(): + job = Job(name='hello_world', role='john_doe', cluster='test-cluster', environment='test', + task=Task(name='main', processes=[], resources=Resources(cpu=0.1, ram=64 * MB, disk=64 * MB)), + production='true', tier='preferred') + with mock.patch('apache.aurora.client.config.deprecation_warning') as mock_warning: + config._validate_deprecated_config(AuroraConfig(job)) + assert mock_warning.call_count == 0