You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by ma...@apache.org on 2015/10/21 20:10:38 UTC

aurora git commit: Adding job update diff details into "aurora job diff" command.

Repository: aurora
Updated Branches:
  refs/heads/master 888f9a3cc -> 5609fc230


Adding job update diff details into "aurora job diff" command.

Bugs closed: AURORA-1516

Reviewed at https://reviews.apache.org/r/39366/


Project: http://git-wip-us.apache.org/repos/asf/aurora/repo
Commit: http://git-wip-us.apache.org/repos/asf/aurora/commit/5609fc23
Tree: http://git-wip-us.apache.org/repos/asf/aurora/tree/5609fc23
Diff: http://git-wip-us.apache.org/repos/asf/aurora/diff/5609fc23

Branch: refs/heads/master
Commit: 5609fc2307892584fbcf6fb66ff134cb3cb6dcf0
Parents: 888f9a3
Author: Maxim Khutornenko <ma...@apache.org>
Authored: Wed Oct 21 11:10:17 2015 -0700
Committer: Maxim Khutornenko <ma...@apache.org>
Committed: Wed Oct 21 11:10:17 2015 -0700

----------------------------------------------------------------------
 .../python/apache/aurora/client/api/__init__.py |  38 +++-
 .../python/apache/aurora/client/cli/jobs.py     | 123 ++++++++---
 .../python/apache/aurora/client/api/test_api.py |   9 +
 .../apache/aurora/client/cli/test_diff.py       | 221 +++++++++++--------
 .../python/apache/aurora/client/cli/util.py     |  40 ++--
 5 files changed, 284 insertions(+), 147 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/5609fc23/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 4b9c48e..5847ca8 100644
--- a/src/main/python/apache/aurora/client/api/__init__.py
+++ b/src/main/python/apache/aurora/client/api/__init__.py
@@ -149,6 +149,18 @@ class AuroraClientAPI(object):
 
     return updater.update(instances)
 
+  def _job_update_request(self, config, instances=None):
+    try:
+      settings = UpdaterConfig(**config.update_config().get()).to_thrift_update_settings(instances)
+    except ValueError as e:
+      raise self.UpdateConfigError(str(e))
+
+    return JobUpdateRequest(
+        instanceCount=config.instances(),
+        settings=settings,
+        taskConfig=config.job().taskConfig
+    )
+
   def start_job_update(self, config, message, instances=None):
     """Requests Scheduler to start job update process.
 
@@ -159,18 +171,8 @@ class AuroraClientAPI(object):
 
     Returns response object with update ID and acquired job lock.
     """
-    try:
-      settings = UpdaterConfig(**config.update_config().get()).to_thrift_update_settings(instances)
-    except ValueError as e:
-      raise self.UpdateConfigError(str(e))
-
+    request = self._job_update_request(config, instances)
     log.info("Starting update for: %s" % config.name())
-    request = JobUpdateRequest(
-        instanceCount=config.instances(),
-        settings=settings,
-        taskConfig=config.job().taskConfig
-    )
-
     return self._scheduler_proxy.startJobUpdate(request, message)
 
   def pause_job_update(self, update_key, message):
@@ -206,6 +208,20 @@ class AuroraClientAPI(object):
     """
     return self._scheduler_proxy.abortJobUpdate(update_key, message)
 
+  def get_job_update_diff(self, config, instances=None):
+    """Requests scheduler to calculate difference between scheduler and client job views.
+
+    Arguments:
+    config -- AuroraConfig instance with update details.
+    message -- Audit message to include with the change.
+    instances -- Optional list of instances to restrict update to.
+
+    Returns response object with job update diff results.
+    """
+    request = self._job_update_request(config, instances)
+    log.debug("Requesting job update diff details for: %s" % config.name())
+    return self._scheduler_proxy.getJobUpdateDiff(request)
+
   def query_job_updates(
       self,
       role=None,

http://git-wip-us.apache.org/repos/asf/aurora/blob/5609fc23/src/main/python/apache/aurora/client/cli/jobs.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/jobs.py b/src/main/python/apache/aurora/client/cli/jobs.py
index 6d15f1e..6dd9dec 100644
--- a/src/main/python/apache/aurora/client/cli/jobs.py
+++ b/src/main/python/apache/aurora/client/cli/jobs.py
@@ -25,6 +25,7 @@ import webbrowser
 from collections import namedtuple
 from copy import deepcopy
 from datetime import datetime
+from itertools import chain
 from pipes import quote
 from tempfile import NamedTemporaryFile
 
@@ -32,6 +33,7 @@ from thrift.protocol import TJSONProtocol
 from thrift.TSerialization import serialize
 
 from apache.aurora.client.api.job_monitor import JobMonitor
+from apache.aurora.client.api.scheduler_client import SchedulerProxy
 from apache.aurora.client.api.updater_util import UpdaterConfig
 from apache.aurora.client.base import get_job_page, synthesize_url
 from apache.aurora.client.cli import (
@@ -157,11 +159,14 @@ class DiffCommand(Verb):
     return "diff"
 
   def get_options(self):
-    return [BIND_OPTION, JSON_READ_OPTION,
+    return [
+        BIND_OPTION,
+        JSON_READ_OPTION,
         CommandOption("--from", dest="rename_from", type=AuroraJobKey.from_path, default=None,
             metavar="cluster/role/env/name",
             help="If specified, the job key to diff against."),
-        JOBSPEC_ARGUMENT, CONFIG_ARGUMENT]
+        INSTANCES_SPEC_ARGUMENT,
+        CONFIG_ARGUMENT]
 
   def pretty_print_task(self, task):
     task.configuration = None
@@ -191,8 +196,52 @@ class DiffCommand(Verb):
     out_file.write("\n")
     out_file.flush()
 
+  def diff_tasks(self, context, local_tasks, remote_tasks):
+    diff_program = os.environ.get("DIFF_VIEWER", "diff")
+    with NamedTemporaryFile() as local:
+      self.dump_tasks(local_tasks, local)
+      with NamedTemporaryFile() as remote:
+        self.dump_tasks(remote_tasks, remote)
+        result = subprocess.call("%s %s %s" % (
+            diff_program, quote(remote.name), quote(local.name)), shell=True)
+        # Unlike most commands, diff doesn't return zero on success; it returns
+        # 1 when a successful diff is non-empty.
+        if result not in (0, 1):
+          raise context.CommandError(EXIT_COMMAND_FAILURE, "Error running diff command")
+
+  def show_diff(self, context, header, configs_summaries, local_task=None):
+    def min_start(ranges):
+      return min(ranges, key=lambda r: r.first).first
+
+    def format_ranges(ranges):
+      instances = []
+      for task_range in sorted(list(ranges), key=lambda r: r.first):
+        if task_range.first == task_range.last:
+          instances.append("[%s]" % task_range.first)
+        else:
+          instances.append("[%s-%s]" % (task_range.first, task_range.last))
+      return instances
+
+    def print_instances(instances):
+      context.print_out("%s %s" % (header, ", ".join(str(span) for span in instances)))
+
+    summaries = sorted(list(configs_summaries), key=lambda s: min_start(s.instances))
+
+    if local_task:
+      for summary in summaries:
+        print_instances(format_ranges(summary.instances))
+        context.print_out("with diff:\n")
+        self.diff_tasks(context, [deepcopy(local_task)], [summary.config])
+        context.print_out('')
+    else:
+      if summaries:
+        print_instances(
+          format_ranges(r for r in chain.from_iterable(s.instances for s in summaries)))
+
   def execute(self, context):
-    config = context.get_job_config(context.options.jobspec, context.options.config_file)
+    config = context.get_job_config(
+        context.options.instance_spec.jobkey,
+        context.options.config_file)
     if context.options.rename_from is not None:
       cluster = context.options.rename_from.cluster
       role = context.options.rename_from.role
@@ -204,34 +253,54 @@ class DiffCommand(Verb):
       env = config.environment()
       name = config.name()
     api = context.get_api(cluster)
-    resp = api.query(api.build_query(role, name, env=env, statuses=ACTIVE_STATES))
-    context.log_response_and_raise(resp, err_code=EXIT_INVALID_PARAMETER,
-        err_msg="Could not find job to diff against")
-    if resp.result.scheduleStatusResult.tasks is None:
-      context.print_err("No tasks found for job %s" % context.options.jobspec)
-      return EXIT_COMMAND_FAILURE
-    else:
-      remote_tasks = [t.assignedTask.task for t in resp.result.scheduleStatusResult.tasks]
+
     resp = api.populate_job_config(config)
-    context.log_response_and_raise(resp, err_code=EXIT_INVALID_CONFIGURATION,
-          err_msg="Error loading configuration")
+    context.log_response_and_raise(
+        resp,
+        err_code=EXIT_INVALID_CONFIGURATION,
+        err_msg="Error loading configuration")
+    local_task = resp.result.populateJobResult.taskConfig
     # Deepcopy is important here as tasks will be modified for printing.
     local_tasks = [
-      deepcopy(resp.result.populateJobResult.taskConfig) for _ in range(config.instances())
+        deepcopy(local_task) for _ in range(config.instances())
     ]
-    diff_program = os.environ.get("DIFF_VIEWER", "diff")
-    with NamedTemporaryFile() as local:
-      self.dump_tasks(local_tasks, local)
-      with NamedTemporaryFile() as remote:
-        self.dump_tasks(remote_tasks, remote)
-        result = subprocess.call("%s %s %s" % (
-            diff_program, quote(remote.name), quote(local.name)), shell=True)
-        # Unlike most commands, diff doesn't return zero on success; it returns
-        # 1 when a successful diff is non-empty.
-        if result not in (0, 1):
-          raise context.CommandError(EXIT_COMMAND_FAILURE, "Error running diff command")
-        else:
-          return EXIT_OK
+
+    def diff_no_update_details():
+      resp = api.query(api.build_query(role, name, env=env, statuses=ACTIVE_STATES))
+      context.log_response_and_raise(
+        resp,
+        err_code=EXIT_INVALID_PARAMETER,
+        err_msg="Could not find job to diff against")
+      if resp.result.scheduleStatusResult.tasks is None:
+        context.print_err("No tasks found for job %s" % context.options.instance_spec.jobkey)
+        return EXIT_COMMAND_FAILURE
+      else:
+        remote_tasks = [t.assignedTask.task for t in resp.result.scheduleStatusResult.tasks]
+      self.diff_tasks(context, local_tasks, remote_tasks)
+
+    if config.raw().has_cron_schedule():
+      diff_no_update_details()
+    else:
+      instances = (None if context.options.instance_spec.instance == ALL_INSTANCES else
+          context.options.instance_spec.instance)
+      try:
+        resp = api.get_job_update_diff(config, instances)
+        context.log_response_and_raise(
+            resp,
+            err_code=EXIT_COMMAND_FAILURE,
+            err_msg="Error getting diff info from scheduler")
+        diff = resp.result.getJobUpdateDiffResult
+        context.print_out("This job update will:")
+        self.show_diff(context, "add instances:", diff.add)
+        self.show_diff(context, "remove instances:", diff.remove)
+        self.show_diff(context, "update instances:", diff.update, local_task)
+        self.show_diff(context, "not change instances:", diff.unchanged)
+      except SchedulerProxy.ThriftInternalError:
+        # TODO(maxim): Temporary fallback to ensure client/scheduler backwards compatibility
+        # (i.e. new client works against old scheduler).
+        diff_no_update_details()
+
+    return EXIT_OK
 
 
 class InspectCommand(Verb):

http://git-wip-us.apache.org/repos/asf/aurora/blob/5609fc23/src/test/python/apache/aurora/client/api/test_api.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/api/test_api.py b/src/test/python/apache/aurora/client/api/test_api.py
index b56e352..7debc79 100644
--- a/src/test/python/apache/aurora/client/api/test_api.py
+++ b/src/test/python/apache/aurora/client/api/test_api.py
@@ -135,6 +135,15 @@ class TestJobUpdateApis(unittest.TestCase):
         self.mock_job_config(error=ValueError()),
         None)
 
+  def test_get_job_update_diff(self):
+    """Test getting job update diff."""
+    api, mock_proxy = self.mock_api()
+    task_config = TaskConfig()
+    mock_proxy.getJobUpdateDiff.return_value = self.create_simple_success_response()
+
+    api.get_job_update_diff(self.mock_job_config(), instances=None)
+    mock_proxy.getJobUpdateDiff.assert_called_once_with(self.create_update_request(task_config))
+
   def test_pause_job_update(self):
     """Test successful job update pause."""
     api, mock_proxy = self.mock_api()

http://git-wip-us.apache.org/repos/asf/aurora/blob/5609fc23/src/test/python/apache/aurora/client/cli/test_diff.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/test_diff.py b/src/test/python/apache/aurora/client/cli/test_diff.py
index 753a041..b9e91cf 100644
--- a/src/test/python/apache/aurora/client/cli/test_diff.py
+++ b/src/test/python/apache/aurora/client/cli/test_diff.py
@@ -14,20 +14,27 @@
 
 import contextlib
 import os
+import textwrap
 
-from mock import Mock, patch
-from twitter.common.contextutil import temporary_file
+from mock import Mock, call, patch
+from pystachio import Empty
 
-from apache.aurora.client.cli import EXIT_INVALID_CONFIGURATION, EXIT_INVALID_PARAMETER
-from apache.aurora.client.cli.client import AuroraCommandLine
+from apache.aurora.client.api import SchedulerProxy
+from apache.aurora.client.cli import EXIT_OK
+from apache.aurora.client.cli.jobs import DiffCommand
+from apache.aurora.client.cli.options import TaskInstanceKey
+from apache.aurora.config import AuroraConfig
+from apache.aurora.config.schema.base import Job
+from apache.thermos.config.schema_base import MB, Process, Resources, Task
 
-from .util import AuroraClientCommandTest
+from .util import AuroraClientCommandTest, FakeAuroraCommandContext, mock_verb_options
 
 from gen.apache.aurora.api.constants import ACTIVE_STATES
 from gen.apache.aurora.api.ttypes import (
-    JobConfiguration,
-    JobKey,
+    ConfigGroup,
+    GetJobUpdateDiffResult,
     PopulateJobResult,
+    Range,
     ResponseCode,
     Result,
     ScheduleStatusResult,
@@ -36,17 +43,30 @@ from gen.apache.aurora.api.ttypes import (
 
 
 class TestDiffCommand(AuroraClientCommandTest):
+  def setUp(self):
+    self._command = DiffCommand()
+    self._mock_options = mock_verb_options(self._command)
+    self._mock_options.instance_spec = TaskInstanceKey(self.TEST_JOBKEY, [0, 1])
+    self._fake_context = FakeAuroraCommandContext()
+    self._fake_context.set_options(self._mock_options)
+    self._mock_api = self._fake_context.get_api("test")
+
   @classmethod
-  def setup_mock_options(cls):
-    """set up to get a mock options object."""
-    mock_options = Mock()
-    mock_options.env = None
-    mock_options.json = False
-    mock_options.bindings = {}
-    mock_options.open_browser = False
-    mock_options.rename_from = None
-    mock_options.cluster = None
-    return mock_options
+  def get_job_config(self, is_cron=False):
+    return AuroraConfig(job=Job(
+      cluster='west',
+      role='bozo',
+      environment='test',
+      name='the_job',
+      service=True if not is_cron else False,
+      cron_schedule='* * * * *' if is_cron else Empty,
+      task=Task(
+        name='task',
+        processes=[Process(cmdline='ls -la', name='process')],
+        resources=Resources(cpu=1.0, ram=1024 * MB, disk=1024 * MB)
+      ),
+      instances=3,
+    ))
 
   @classmethod
   def create_status_response(cls):
@@ -60,93 +80,102 @@ class TestDiffCommand(AuroraClientCommandTest):
     return cls.create_blank_response(ResponseCode.INVALID_REQUEST, 'No tasks found for query')
 
   @classmethod
-  def setup_populate_job_config(cls, api):
+  def populate_job_config_result(cls):
     populate = cls.create_simple_success_response()
-    api.populateJobConfig.return_value = populate
     populate.result = Result(populateJobResult=PopulateJobResult(
         taskConfig=cls.create_scheduled_tasks()[0].assignedTask.task))
     return populate
 
-  def test_successful_diff(self):
-    """Test the diff command."""
-    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
+  @classmethod
+  def get_job_update_diff_result(cls):
+    diff = cls.create_simple_success_response()
+    task = cls.create_task_config('foo')
+    diff.result = Result(getJobUpdateDiffResult=GetJobUpdateDiffResult(
+        add=set([ConfigGroup(
+            config=task,
+            instances=frozenset([Range(first=10, last=10), Range(first=12, last=14)]))]),
+        remove=frozenset(),
+        update=frozenset([ConfigGroup(
+            config=task,
+            instances=frozenset([Range(first=11, last=11)]))]),
+        unchanged=frozenset([ConfigGroup(
+            config=task,
+            instances=frozenset([Range(first=0, last=9)]))])
+    ))
+    return diff
+
+  def test_service_diff(self):
+    config = self.get_job_config()
+    self._fake_context.get_job_config = Mock(return_value=config)
+    self._mock_api.populate_job_config.return_value = self.populate_job_config_result()
+    self._mock_api.get_job_update_diff.return_value = self.get_job_update_diff_result()
+
     with contextlib.nested(
-        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
         patch('subprocess.call', return_value=0),
-        patch('json.loads', return_value={})) as (_, subprocess_patch, _):
-
-      mock_scheduler_proxy.getTasksStatus.return_value = self.create_status_response()
-      self.setup_populate_job_config(mock_scheduler_proxy)
-      with temporary_file() as fp:
-        fp.write(self.get_valid_config())
-        fp.flush()
-        cmd = AuroraCommandLine()
-        cmd.execute(['job', 'diff', 'west/bozo/test/hello', fp.name])
-
-        # Diff should get the task status, populate a config, and run diff.
-        mock_scheduler_proxy.getTasksStatus.assert_called_with(
-            TaskQuery(jobKeys=[JobKey(role='bozo', environment='test', name='hello')],
-                      statuses=ACTIVE_STATES))
-        assert mock_scheduler_proxy.populateJobConfig.call_count == 1
-        assert isinstance(mock_scheduler_proxy.populateJobConfig.call_args[0][0], JobConfiguration)
-        assert (mock_scheduler_proxy.populateJobConfig.call_args[0][0].key ==
-            JobKey(environment=u'test', role=u'bozo', name=u'hello'))
-        # Subprocess should have been used to invoke diff.
-        assert subprocess_patch.call_count == 1
-        assert subprocess_patch.call_args[0][0].startswith(
-            os.environ.get('DIFF_VIEWER', 'diff') + ' ')
-
-  def test_diff_invalid_config(self):
-    """Test the diff command if the user passes a config with an error in it."""
-    mock_options = self.setup_mock_options()
-    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
-    mock_scheduler_proxy.getTasksStatus.return_value = self.create_status_response()
-    self.setup_populate_job_config(mock_scheduler_proxy)
+        patch('json.loads', return_value={})) as (subprocess_patch, _):
+
+      result = self._command.execute(self._fake_context)
+
+      assert result == EXIT_OK
+      assert self._mock_api.populate_job_config.mock_calls == [call(config)]
+      assert self._mock_api.get_job_update_diff.mock_calls == [
+          call(config, self._mock_options.instance_spec.instance)
+      ]
+      assert "\n".join(self._fake_context.get_out()) == textwrap.dedent("""\
+        This job update will:
+        add instances: [10], [12-14]
+        update instances: [11]
+        with diff:\n\n
+        not change instances: [0-9]""")
+      assert subprocess_patch.call_count == 1
+      assert subprocess_patch.call_args[0][0].startswith(
+          os.environ.get('DIFF_VIEWER', 'diff') + ' ')
+
+  def test_service_diff_old_api(self):
+    config = self.get_job_config()
+    query = TaskQuery(
+        jobKeys=[self.TEST_JOBKEY.to_thrift()],
+        statuses=ACTIVE_STATES)
+    self._fake_context.get_job_config = Mock(return_value=config)
+    self._mock_api.populate_job_config.return_value = self.populate_job_config_result()
+    self._mock_api.get_job_update_diff.side_effect = SchedulerProxy.ThriftInternalError("Expected")
+    self._mock_api.query.return_value = self.create_empty_task_result()
+    self._mock_api.build_query.return_value = query
+
     with contextlib.nested(
-        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
-        patch('twitter.common.app.get_options', return_value=mock_options),
         patch('subprocess.call', return_value=0),
-        patch('json.loads', return_value=Mock())) as (
-            mock_scheduler_proxy_class,
-            options,
-            subprocess_patch,
-            json_patch):
-      with temporary_file() as fp:
-        fp.write(self.get_invalid_config('stupid="me"',))
-        fp.flush()
-        cmd = AuroraCommandLine()
-        result = cmd.execute(['job', 'diff', 'west/bozo/test/hello', fp.name])
-        assert result == EXIT_INVALID_CONFIGURATION
-        assert mock_scheduler_proxy.getTasksStatus.call_count == 0
-        assert mock_scheduler_proxy.populateJobConfig.call_count == 0
-        assert subprocess_patch.call_count == 0
-
-  def test_diff_server_error(self):
-    """Test the diff command if the user passes a config with an error in it."""
-    mock_options = self.setup_mock_options()
-    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
-    mock_scheduler_proxy.getTasksStatus.return_value = self.create_failed_status_response()
-    self.setup_populate_job_config(mock_scheduler_proxy)
+        patch('json.loads', return_value={})) as (subprocess_patch, _):
+
+      result = self._command.execute(self._fake_context)
+      assert result == EXIT_OK
+      assert self._mock_api.populate_job_config.mock_calls == [call(config)]
+      assert self._mock_api.get_job_update_diff.mock_calls == [
+          call(config, self._mock_options.instance_spec.instance)
+      ]
+      assert self._mock_api.query.mock_calls == [call(query)]
+      assert subprocess_patch.call_count == 1
+      assert subprocess_patch.call_args[0][0].startswith(
+          os.environ.get('DIFF_VIEWER', 'diff') + ' ')
+
+  def test_cron_diff(self):
+    config = self.get_job_config(is_cron=True)
+    query = TaskQuery(
+        jobKeys=[self.TEST_JOBKEY.to_thrift()],
+        statuses=ACTIVE_STATES)
+    self._fake_context.get_job_config = Mock(return_value=config)
+    self._mock_api.populate_job_config.return_value = self.populate_job_config_result()
+    self._mock_api.query.return_value = self.create_empty_task_result()
+    self._mock_api.build_query.return_value = query
+
     with contextlib.nested(
-        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
-        patch('twitter.common.app.get_options', return_value=mock_options),
         patch('subprocess.call', return_value=0),
-        patch('json.loads', return_value=Mock())) as (
-            mock_scheduler_proxy_class,
-            options,
-            subprocess_patch,
-            json_patch):
-
-      with temporary_file() as fp:
-        fp.write(self.get_valid_config())
-        fp.flush()
-        cmd = AuroraCommandLine()
-        result = cmd.execute(['job', 'diff', 'west/bozo/test/hello', fp.name])
-        assert result == EXIT_INVALID_PARAMETER
-        # In this error case, we should have called the server getTasksStatus;
-        # but since it fails, we shouldn't call populateJobConfig or subprocess.
-        mock_scheduler_proxy.getTasksStatus.assert_called_with(
-            TaskQuery(jobKeys=[JobKey(role='bozo', environment='test', name='hello')],
-                statuses=ACTIVE_STATES))
-        assert mock_scheduler_proxy.populateJobConfig.call_count == 0
-        assert subprocess_patch.call_count == 0
+        patch('json.loads', return_value={})) as (subprocess_patch, _):
+
+      result = self._command.execute(self._fake_context)
+
+      assert result == EXIT_OK
+      assert self._mock_api.populate_job_config.mock_calls == [call(config)]
+      assert self._mock_api.query.mock_calls == [call(query)]
+      assert subprocess_patch.call_count == 1
+      assert subprocess_patch.call_args[0][0].startswith(
+          os.environ.get('DIFF_VIEWER', 'diff') + ' ')

http://git-wip-us.apache.org/repos/asf/aurora/blob/5609fc23/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 b03148b..4b5ef4d 100644
--- a/src/test/python/apache/aurora/client/cli/util.py
+++ b/src/test/python/apache/aurora/client/cli/util.py
@@ -50,11 +50,21 @@ def mock_verb_options(verb):
   def opt_name(opt):
     return opt.name.lstrip('--').replace('-', '_')
 
-  options = Mock(spec_set=[opt_name(opt) for opt in verb.get_options()])
+  def name_or_dest(opt):
+    """Prefers 'dest' if available otherwise defaults to name."""
+    return opt.kwargs.get('dest') if 'dest' in opt.kwargs else opt_name(opt)
+
+  options = Mock(
+    spec_set=[name_or_dest(opt) for opt in verb.get_options()]
+  )
+
   # Apply default values to options.
   for opt in verb.get_options():
     if 'default' in opt.kwargs:
-      setattr(options, opt_name(opt), opt.kwargs.get('default'))
+      setattr(
+          options,
+          name_or_dest(opt),
+          opt.kwargs.get('default'))
   return options
 
 
@@ -204,6 +214,20 @@ class AuroraClientCommandTest(unittest.TestCase):
     return task
 
   @classmethod
+  def create_task_config(cls, name):
+    return TaskConfig(
+        maxTaskFailures=1,
+        executorConfig=ExecutorConfig(data='fake data'),
+        metadata=[],
+        job=JobKey(role=cls.TEST_ROLE, environment=cls.TEST_ENV, name=name),
+        owner=Identity(role=cls.TEST_ROLE),
+        environment=cls.TEST_ENV,
+        jobName=name,
+        numCpus=2,
+        ramMb=2,
+        diskMb=2)
+
+  @classmethod
   def create_scheduled_tasks(cls):
     tasks = []
     for name in ['foo', 'bar', 'baz']:
@@ -212,17 +236,7 @@ class AuroraClientCommandTest(unittest.TestCase):
           assignedTask=AssignedTask(
               taskId=1287391823,
               slaveHost='slavehost',
-              task=TaskConfig(
-                  maxTaskFailures=1,
-                  executorConfig=ExecutorConfig(data='fake data'),
-                  metadata=[],
-                  job=JobKey(role=cls.TEST_ROLE, environment=cls.TEST_ENV, name=name),
-                  owner=Identity(role=cls.TEST_ROLE),
-                  environment=cls.TEST_ENV,
-                  jobName=name,
-                  numCpus=2,
-                  ramMb=2,
-                  diskMb=2),
+              task=cls.create_task_config(name),
               instanceId=4237894,
               assignedPorts={}),
           status=ScheduleStatus.RUNNING,