You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by jc...@apache.org on 2016/07/08 20:33:45 UTC

aurora git commit: AURORA-1710 Make 'tier' required and remove support for 'production' flag in Job configuration - CLI changes

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 <me...@nurolahzade.com>
Authored: Fri Jul 8 15:33:24 2016 -0500
Committer: Joshua Cohen <jc...@apache.org>
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