You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by mc...@apache.org on 2014/07/15 20:11:38 UTC

git commit: Improve aurora "job diff" command.

Repository: incubator-aurora
Updated Branches:
  refs/heads/master 62954a2cf -> 92799eae2


Improve aurora "job diff" command.

Add a new diff method, which uses field-by-field comparison of JSON trees for comparing running job configurations to potentially updated configs.

- Allow exclusion of semantically irrelevant fields.
- Provide a clearer list of the differences between configs.
- Provide a scripting-friendly alternative JSON syntax for diffs.

The old diff behavior is still available under the "--use-shell-diff" option.

Testing Done:
New unit tests of the JSON tree diff code, plus a bunch of new "job diff" tests of the new functionality.
All tests pass.

Bugs closed: aurora-520

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


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

Branch: refs/heads/master
Commit: 92799eae2819cb81eba89ba6201f1ca333cfd503
Parents: 62954a2
Author: Mark Chu-Carroll <mc...@twopensource.com>
Authored: Tue Jul 15 14:08:27 2014 -0400
Committer: Mark Chu-Carroll <mc...@twitter.com>
Committed: Tue Jul 15 14:08:27 2014 -0400

----------------------------------------------------------------------
 src/main/python/apache/aurora/client/cli/BUILD  |   1 +
 .../python/apache/aurora/client/cli/jobs.py     | 131 +++++-
 .../apache/aurora/client/cli/json_tree_diff.py  | 116 ++++++
 src/test/python/apache/aurora/client/cli/BUILD  |   7 +
 .../apache/aurora/client/cli/test_diff.py       | 412 ++++++++++++++++---
 .../apache/aurora/client/cli/test_json_diff.py  | 100 +++++
 6 files changed, 701 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/92799eae/src/main/python/apache/aurora/client/cli/BUILD
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/BUILD b/src/main/python/apache/aurora/client/cli/BUILD
index ebe681a..1ffa603 100644
--- a/src/main/python/apache/aurora/client/cli/BUILD
+++ b/src/main/python/apache/aurora/client/cli/BUILD
@@ -54,6 +54,7 @@ python_library(
     'command_hooks.py',
     'cron.py',
     'jobs.py',
+    'json_tree_diff.py',
     'logsetup.py',
     'options.py',
     'quota.py',

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/92799eae/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 4fa03a6..295cf9a 100644
--- a/src/main/python/apache/aurora/client/cli/jobs.py
+++ b/src/main/python/apache/aurora/client/cli/jobs.py
@@ -21,10 +21,10 @@ import pprint
 import subprocess
 import time
 from datetime import datetime
-from tempfile import NamedTemporaryFile
 
 from thrift.protocol import TJSONProtocol
 from thrift.TSerialization import serialize
+from twitter.common.contextutil import temporary_file
 
 from apache.aurora.client.api.job_monitor import JobMonitor
 from apache.aurora.client.api.updater_util import UpdaterConfig
@@ -38,6 +38,7 @@ from apache.aurora.client.cli import (
     Verb
 )
 from apache.aurora.client.cli.context import AuroraCommandContext
+from apache.aurora.client.cli.json_tree_diff import compare_pruned_json
 from apache.aurora.client.cli.options import (
     ALL_INSTANCES,
     BATCH_OPTION,
@@ -137,8 +138,20 @@ class DiffCommand(Verb):
   @property
   def help(self):
     return """Compare a job configuration against a running job.
-By default the diff will be displayed using 'diff', though you may choose an
-alternate diff program by setting the DIFF_VIEWER environment variable."""
+
+Prints a list of the changes between the local configuration, and the remote
+executing job spec.
+
+If the "--write-json" option is passed, then this will print the differences
+between the deployed and local configuration in JSON format as a list containing
+one dict for each difference.
+
+The dicts contain a field "difftype" which identifies the type of difference described
+by that dict. If the lengths of the task lists differ, then there will be a single
+record: {"difftype": "num_tasks", "remote" #, "local": #}. If fields of corresponding
+tasks differ, there will be a record {"difftype": "fields", "task": #, [ field_diffs ]}
+where each field_diff is: {"field": fieldname, "local": value, "remote": value}
+"""
 
   @property
   def name(self):
@@ -146,10 +159,25 @@ alternate diff program by setting the DIFF_VIEWER environment variable."""
 
   def get_options(self):
     return [BIND_OPTION, JSON_READ_OPTION,
+        JSON_WRITE_OPTION,
         CommandOption("--from", dest="rename_from", type=AuroraJobKey.from_path, default=None,
             help="If specified, the job key to diff against."),
+        CommandOption("--use-shell-diff", default=False, action="store_true",
+            help=("If specified, write the configs to disk, and use DIFF_VIEWER or unix diff to "
+                " compare them")),
+        CommandOption("--exclude-field", default=[], action="append",
+            help=("Path expression for task config fields that should be skipped in comparison\n"
+                "  (applies to tree diff only)")),
         JOBSPEC_ARGUMENT, CONFIG_ARGUMENT]
 
+  def get_task_json(self, task):
+    task.configuration = None
+    task.executorConfig = ExecutorConfig(name=AURORA_EXECUTOR_NAME,
+        data=json.loads(task.executorConfig.data))
+    data = serialize(task, protocol_factory=TJSONProtocol.TSimpleJSONProtocolFactory())
+
+    return json.loads(data)
+
   def pretty_print_task(self, task):
     task.configuration = None
     task.executorConfig = ExecutorConfig(
@@ -165,6 +193,84 @@ alternate diff program by setting the DIFF_VIEWER environment variable."""
     out_file.write("\n")
     out_file.flush()
 
+  def do_shell_diff(self, context, local_tasks, remote_tasks):
+    """Compute diffs externally, using a unix diff program"""
+    diff_program = os.environ.get("DIFF_VIEWER", "diff")
+    with temporary_file() as local:
+      self.dump_tasks(local_tasks, local)
+      with temporary_file() as remote:
+        self.dump_tasks(remote_tasks, remote)
+        result = subprocess.call([diff_program, remote.name, local.name])
+        # 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 _parse_excludes_parameters(self, context):
+    result = []
+    for f in context.options.exclude_field:
+      path = f.split(".")
+      result.append(path)
+    return result
+
+  def do_json_diff(self, context, local_tasks, remote_tasks):
+    """Compute diffs internally, based on the JSON tree form of the task configs."""
+    # String constants, for generating JSON
+    DIFFTYPE = "difftype"
+    NUMTASKS = "num_tasks"
+    LOCAL = "local"
+    REMOTE = "remote"
+    FIELDS = "fields"
+    FIELD = "field"
+    TASK = "task"
+
+    write_json = context.options.write_json
+    found_diffs = 0
+    json_out = []
+    # Compare lengths
+    if len(local_tasks) != len(remote_tasks):
+      found_diffs += abs(len(local_tasks) - len(remote_tasks))
+      if write_json:
+        json_out.append({DIFFTYPE: NUMTASKS,
+            LOCAL: len(local_tasks), REMOTE: len(remote_tasks)})
+      else:
+        context.print_out("Local config has a different number of tasks: %s local vs %s running" %
+            (len(local_tasks), len(remote_tasks)))
+
+    # Compare each instance, excluding the Identity.user:
+    excludes = [["owner", "user"]] + self._parse_excludes_parameters(context)
+    for i in range(min(len(local_tasks), len(remote_tasks))):
+      local_task = self.get_task_json(local_tasks[i])
+      remote_task = self.get_task_json(remote_tasks[i])
+      task_diffs = compare_pruned_json(local_task, remote_task, excludes)
+      if len(task_diffs) > 0:
+        if write_json:
+          json_task_diffs = {DIFFTYPE: FIELDS, TASK: i}
+          field_diffs = []
+          for task_diff in task_diffs:
+            field_diffs.append({FIELD: task_diff.name,
+                LOCAL: task_diff.base, REMOTE: task_diff.other})
+            found_diffs += 1
+          json_task_diffs[FIELDS] = field_diffs
+          json_out.append(json_task_diffs)
+        else:
+          context.print_out("Task diffs found in instance %s" % i)
+          for task_diff in task_diffs:
+            context.print_out("\tField '%s' is '%s' local, but '%s' remote" %
+                (task_diff.name, task_diff.base, task_diff.other))
+            found_diffs += 1
+
+    if write_json:
+      context.print_out(json.dumps(json_out, indent=2, separators=[",", ": "], sort_keys=False))
+    else:
+      if found_diffs > 0:
+        context.print_out("%s total diff(s) found" % found_diffs)
+      else:
+        context.print_out("No diffs found!")
+    return EXIT_OK
+
   def execute(self, context):
     config = context.get_job_config(context.options.jobspec, context.options.config_file)
     if context.options.rename_from is not None:
@@ -177,27 +283,22 @@ alternate diff program by setting the DIFF_VIEWER environment variable."""
       role = config.role()
       env = config.environment()
       name = config.name()
+
     api = context.get_api(cluster)
     resp = api.query(api.build_query(role, name, statuses=ACTIVE_STATES, env=env))
     context.check_and_log_response(resp, err_code=EXIT_INVALID_PARAMETER,
         err_msg="Could not find job to diff against")
     remote_tasks = [t.assignedTask.task for t in resp.result.scheduleStatusResult.tasks]
+    # The remote_tasks are a list of TaskConfigs.
     resp = api.populate_job_config(config)
     context.check_and_log_response(resp, err_code=EXIT_INVALID_CONFIGURATION,
           err_msg="Error loading configuration; see log for details")
     local_tasks = resp.result.populateJobResult.populated
-    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([diff_program, remote.name, local.name])
-        # 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
+    if context.options.use_shell_diff:
+      return self.do_shell_diff(context, local_tasks, remote_tasks)
+    else:
+      return self.do_json_diff(context, local_tasks, remote_tasks)
+
 
 
 class InspectCommand(Verb):

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/92799eae/src/main/python/apache/aurora/client/cli/json_tree_diff.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/json_tree_diff.py b/src/main/python/apache/aurora/client/cli/json_tree_diff.py
new file mode 100644
index 0000000..0b57383
--- /dev/null
+++ b/src/main/python/apache/aurora/client/cli/json_tree_diff.py
@@ -0,0 +1,116 @@
+#
+# 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.
+#
+
+#
+# Code to compute field-wise tree diffs over json trees, with exclusions.
+#
+
+from collections import namedtuple
+
+from twitter.common.lang import Compatibility
+
+
+def prune_structure(struct, exclusions):
+  """Remove fields specified in an exclusion from a json dictionary."""
+  result = dict(struct)
+  for e in exclusions:
+    if isinstance(e, list) and len(e) == 1 and isinstance(e[0], Compatibility.string):
+      e = e[0]
+    if isinstance(e, Compatibility.string) and e in struct:
+      del result[e]
+    else:
+      first = e[0]
+      if first in struct:
+        result[first] = prune_structure(result[first], [e[1:]])
+  return result
+
+
+FieldDifference = namedtuple("FieldDifference", ["name", "base", "other"])
+
+
+def compare_json(base, other, path):
+  """Do a field-wise comparison between two json trees.
+  :param base: one of the JSON trees to compare
+  :param other: the other JSON tree
+  :param path: a list of string describing the path to the trees being
+      compared (used for labelling the diff.)"""
+
+  # JSON sets are converted to JSON as lists, but the order of elements in the list
+  # isn't consistent - it's possible to list-ify the elements of two identical sets,
+  # and get lists with the elements in different orders. So we need to fix that.
+  #
+  # For the configs that we'll be comparing with this code, all of the collections
+  # are lists, which means that we can just sort the elements of any list, and then
+  # do a list comparison.
+  #
+  # But for the future, we should really find a better solution, like fixing the
+  # code that converts thrift to JSON in order to make it generate sets consistently.
+
+  def canonicalize(val):
+    def canonicalize_list(lst):
+      result = []
+      for l in lst:
+        result.append(canonicalize(l))
+      result.sort()
+      return result
+
+    def canonicalize_dict(dct):
+      result = {}
+      for key in dct:
+        result[key] = canonicalize(dct[key])
+      return result
+
+    if isinstance(val, list):
+      return canonicalize_list(val)
+    elif isinstance(val, dict):
+      return canonicalize_dict(val)
+    elif isinstance(val, set):
+      return canonicalize_list(set)
+    else:
+      return val
+
+  keys = set(base.keys()) | set(other.keys())
+  differences = []
+  for key in keys:
+    base_val = canonicalize(base.get(key, "__undefined__"))
+    other_val = canonicalize(other.get(key, "__undefined__"))
+    if base_val != other_val:
+      if isinstance(base_val, dict) and isinstance(other_val, dict):
+        differences += compare_json(base_val, other_val, path + [key])
+      else:
+        differences += [FieldDifference('.'.join(path + [key]), base_val, other_val)]
+  return differences
+
+
+def compare_pruned_json(base, other, excludes):
+  """Compares two thrift objects, which have been rendered as JSON dictionaries.
+
+  The two are considered equal if the fields outside of the excludes list are
+  equal.
+  :param base: one version of the thrift object; assumed to be the original, unmodified structure.
+  :param other: a second version of the thrift object, assumed to be a possibly modified copy
+       of the base.
+  :param excludes: a structured list of fields that should not be considered in the comparison.
+      Each element is either a string or a list.
+       - If an entry is a string, then it is the name of a field in the object whose value
+         should not be considered in the comparison.
+       - If an entry is a list [x, y, z], then it is interpreted as a path specification
+         for a field in a nested object. The list [x, y, z] would mean "ignore the field
+         z of the object that is in the field y of the object in the field x of the two
+         objects being compared."
+         ["x"] and "x" are always equivalent.
+  """
+  pruned_base = prune_structure(base, excludes)
+  pruned_other = prune_structure(other, excludes)
+  return compare_json(pruned_base, pruned_other, [])

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/92799eae/src/test/python/apache/aurora/client/cli/BUILD
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/BUILD b/src/test/python/apache/aurora/client/cli/BUILD
index 3c88ed7..f12cd52 100644
--- a/src/test/python/apache/aurora/client/cli/BUILD
+++ b/src/test/python/apache/aurora/client/cli/BUILD
@@ -21,6 +21,7 @@ python_test_suite(
     pants(':cron'),
     pants(':help'),
     pants(':job'),
+    pants(':json_diff'),
     pants(':config'),
     pants(':logging'),
     pants(':plugins'),
@@ -30,6 +31,12 @@ python_test_suite(
   ]
 )
 
+python_tests(
+  name='json_diff',
+  sources = [ 'test_json_diff.py' ],
+  dependencies = [ pants('src/main/python/apache/aurora/client/cli:client_lib') ]
+)
+
 python_library(
   name = 'util',
   sources = [ 'util.py' ],

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/92799eae/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 38629b6..ab64e4f 100644
--- a/src/test/python/apache/aurora/client/cli/test_diff.py
+++ b/src/test/python/apache/aurora/client/cli/test_diff.py
@@ -13,12 +13,13 @@
 #
 
 import contextlib
+import json
 import os
 
 from mock import Mock, patch
 from twitter.common.contextutil import temporary_file
 
-from apache.aurora.client.cli import EXIT_INVALID_CONFIGURATION, EXIT_INVALID_PARAMETER
+from apache.aurora.client.cli import EXIT_INVALID_CONFIGURATION, EXIT_INVALID_PARAMETER, EXIT_OK
 from apache.aurora.client.cli.client import AuroraCommandLine
 from apache.aurora.client.cli.util import AuroraClientCommandTest
 
@@ -29,8 +30,12 @@ from gen.apache.aurora.api.ttypes import (
     Identity,
     JobConfiguration,
     JobKey,
+    Metadata,
     PopulateJobResult,
+    Response,
     ResponseCode,
+    Result,
+    ScheduledTask,
     ScheduleStatus,
     ScheduleStatusResult,
     TaskConfig,
@@ -38,6 +43,29 @@ from gen.apache.aurora.api.ttypes import (
     TaskQuery
 )
 
+MOCK_LOG = []
+
+
+def mock_log(*args):
+  MOCK_LOG.append(args)
+
+
+def clear_mock_log():
+  global MOCK_LOG
+  MOCK_LOG = []
+
+
+MOCK_OUT = []
+
+
+def mock_out(s):
+  MOCK_OUT.append(s)
+
+
+def clear_mock_out():
+  global MOCK_OUT
+  MOCK_OUT = []
+
 
 class TestDiffCommand(AuroraClientCommandTest):
   @classmethod
@@ -54,41 +82,72 @@ class TestDiffCommand(AuroraClientCommandTest):
     return mock_options
 
   @classmethod
-  def create_mock_scheduled_tasks(cls):
-    jobs = []
-    for name in ['foo', 'bar', 'baz']:
-      job = Mock()
-      job.key = JobKey(role=cls.TEST_ROLE, environment=cls.TEST_ENV, name=name)
-      job.failure_count = 0
-      job.assignedTask = Mock(spec=AssignedTask)
-      job.assignedTask.slaveHost = 'slavehost'
-      job.assignedTask.task = Mock(spec=TaskConfig)
-      job.assignedTask.task.maxTaskFailures = 1
-      job.assignedTask.task.executorConfig = Mock(spec=ExecutorConfig)
-      job.assignedTask.task.executorConfig.data = Mock()
-      job.assignedTask.task.metadata = []
-      job.assignedTask.task.owner = Identity(role='bozo')
-      job.assignedTask.task.environment = 'test'
-      job.assignedTask.task.jobName = 'woops'
-      job.assignedTask.task.numCpus = 2
-      job.assignedTask.task.ramMb = 2
-      job.assignedTask.task.diskMb = 2
-      job.assignedTask.instanceId = 4237894
-      job.assignedTask.assignedPorts = None
-      job.status = ScheduleStatus.RUNNING
-      mockEvent = Mock(spec=TaskEvent)
-      mockEvent.timestamp = 28234726395
-      mockEvent.status = ScheduleStatus.RUNNING
-      mockEvent.message = "Hi there"
-      job.taskEvents = [mockEvent]
-      jobs.append(job)
-    return jobs
+  def create_mock_scheduled_task(cls, task_name, max_failures, num_cpus, role, metadata):
+    task = ScheduledTask()
+    task.key = JobKey(role=cls.TEST_ROLE, environment=cls.TEST_ENV, name=task_name)
+    task.failure_count = 0
+    task.assignedTask = Mock(spec=AssignedTask)
+    task.assignedTask.slaveHost = 'slavehost'
+    task.assignedTask.task = TaskConfig()
+    task.assignedTask.task.maxTaskFailures = max_failures
+    task.assignedTask.task.executorConfig = ExecutorConfig()
+    task.assignedTask.task.executorConfig.data = '[]'
+    task.assignedTask.task.metadata = metadata
+    task.assignedTask.task.owner = Identity(role=role)
+    task.assignedTask.task.environment = 'test'
+    task.assignedTask.task.jobName = task_name
+    task.assignedTask.task.numCpus = num_cpus
+    task.assignedTask.task.ramMb = 2
+    task.assignedTask.task.diskMb = 2
+    task.assignedTask.instanceId = 4237894
+    task.assignedTask.assignedPorts = None
+    task.status = ScheduleStatus.RUNNING
+    mockEvent = Mock(spec=TaskEvent)
+    mockEvent.timestamp = 28234726395
+    mockEvent.status = ScheduleStatus.RUNNING
+    mockEvent.message = "Hi there"
+    task.taskEvents = [mockEvent]
+    return task
+
+  @classmethod
+  def create_mock_scheduled_tasks(cls, task_specs=None):
+    tasks = []
+    if task_specs is None:
+      task_specs = [{'name': 'foo'}, {'name': 'bar'}, {'name': 'baz'}]
+    for task_spec in task_specs:
+      task = cls.create_mock_scheduled_task(task_spec["name"],
+          task_spec.get("max_failures", 1),
+          num_cpus=task_spec.get("num_cpus", 2),
+          role=task_spec.get("role", "bozo"),
+          metadata=task_spec.get("metadata", []))
+      tasks.append(task)
+    return tasks
+
+  @classmethod
+  def create_mock_taskconfigs(cls, task_specs=None):
+    tasks = []
+    if task_specs is None:
+      task_specs = [{'name': 'foo'}, {'name': 'bar'}, {'name': 'baz'}]
+    for task_spec in task_specs:
+      task = TaskConfig()
+      task.maxTaskFailures = task_spec.get("max_task_failures", 1)
+      task.executorConfig = ExecutorConfig()
+      task.executorConfig.data = '[]'
+      task.metadata = task_spec.get("metadata", [])
+      task.owner = Identity(role=task_spec.get("role", "bozo"))
+      task.environment = 'test'
+      task.jobName = task_spec['name']
+      task.numCpus = task_spec.get("num_cpus", 2)
+      task.ramMb = 2
+      task.diskMb = 2
+      tasks.append(task)
+    return tasks
 
   @classmethod
-  def create_status_response(cls):
+  def create_status_response(cls, specs=None):
     resp = cls.create_simple_success_response()
     resp.result.scheduleStatusResult = Mock(spec=ScheduleStatusResult)
-    resp.result.scheduleStatusResult.tasks = set(cls.create_mock_scheduled_tasks())
+    resp.result.scheduleStatusResult.tasks = set(cls.create_mock_scheduled_tasks(specs))
     return resp
 
   @classmethod
@@ -96,28 +155,255 @@ class TestDiffCommand(AuroraClientCommandTest):
     return cls.create_blank_response(ResponseCode.INVALID_REQUEST, 'No tasks found for query')
 
   @classmethod
-  def setup_populate_job_config(cls, api):
-    populate = cls.create_simple_success_response()
-    populate.result.populateJobResult = Mock(spec=PopulateJobResult)
+  def setup_populate_job_config(cls, api, task_specs=None):
+    populate = Response()
+    populate.responseCode = ResponseCode.OK
+    populate.messageDEPRECATED = "Ok"
+    populate.result = Result()
+    populate.result.populateJobResult = PopulateJobResult()
+    populate.result.populateJobResult.populated = cls.create_mock_taskconfigs(task_specs)
     api.populateJobConfig.return_value = populate
-    populate.result.populateJobResult.populated = cls.create_mock_scheduled_tasks()
     return populate
 
-  def test_successful_diff(self):
-    """Test the diff command."""
+  def test_success_no_diffs(self):
+    result = self._test_successful_diff_generic(None, None)
+    assert MOCK_OUT == ["No diffs found!"]
+    assert result == EXIT_OK
+
+  def test_success_no_diffs_metadata(self):
+    # Metadata in different order, but same data.
+    one = [{"name": "serv", "metadata": [Metadata(key="a", value="1"),
+        Metadata(key="b", value="2"), Metadata(key="instance", value="0")]},
+        {"name": "serv", "metadata": [Metadata(key="a", value="1"),
+            Metadata(key="b", value="2"), Metadata(key="instance", value="1")]},
+        {"name": "serv", "metadata": [Metadata(key="a", value="1"),
+            Metadata(key="b", value="2"), Metadata(key="instance", value="2")]}]
+
+    two = [{"name": "serv", "metadata": [Metadata(key="b", value="2"),
+        Metadata(key="a", value="1"), Metadata(key="instance", value="0")]},
+        {"name": "serv", "metadata": [Metadata(key="instance", value="1"),
+        Metadata(key="a", value="1"), Metadata(key="b", value="2")]},
+        {"name": "serv", "metadata": [Metadata(key="a", value="1"),
+        Metadata(key="instance", value="2"), Metadata(key="b", value="2")]}]
+
+    result = self._test_successful_diff_generic(one, two)
+    assert result == EXIT_OK
+    assert MOCK_OUT == ["No diffs found!"]
+
+  def test_success_diffs_metadata(self):
+    one = [{"name": "serv", "metadata": [Metadata(key="a", value="1"),
+        Metadata(key="b", value="2"), Metadata(key="instance", value="0")]},
+        {"name": "serv", "metadata": [Metadata(key="a", value="1"),
+            Metadata(key="b", value="2"), Metadata(key="instance", value="1")]},
+        {"name": "serv", "metadata": [Metadata(key="a", value="1"),
+            Metadata(key="b", value="2"), Metadata(key="instance", value="2")]}]
+
+    two = [{"name": "serv", "metadata": [Metadata(key="b", value="2"),
+        Metadata(key="a", value="1"), Metadata(key="instance", value="0")]},
+        {"name": "serv", "metadata": [Metadata(key="instance", value="1"),
+        Metadata(key="a", value="3"), Metadata(key="b", value="2")]},
+        {"name": "serv", "metadata": [Metadata(key="a", value="1"),
+        Metadata(key="instance", value="2"), Metadata(key="b", value="2")]}]
+
+    result = self._test_successful_diff_generic(one, two)
+    assert result == EXIT_OK
+    print(MOCK_OUT)
+    assert MOCK_OUT == ['Task diffs found in instance 1',
+        (u"\tField 'metadata' is '[{u'key': u'a', u'value': u'3'}, {u'key': u'b', u'value': u'2'}, "
+             "{u'key': u'instance', u'value': u'1'}]' local, but '[{u'key': u'a', u'value': u'1'}, "
+            "{u'key': u'b', u'value': u'2'}, {u'key': u'instance', u'value': u'1'}]' remote"),
+        '1 total diff(s) found']
+
+  def test_success_no_diffs_json(self):
+    result = self._test_successful_diff_generic(None, None, write_json=True)
+    # No diffs, in json, shows as an empty list of diffs
+    assert MOCK_OUT == ["[]"]
+    assert result == EXIT_OK
+
+  def test_success_with_diffs_one(self):
+    # owner.role different in task 0
+    result = self._test_successful_diff_generic([{"name": "foo", "role": "me"},
+            {"name": "bar", "role": "you"}, {"name": "baz", "role": "you"}],
+        [{"name": "foo", "role": "you"}, {"name": "bar", "role": "you"},
+            {"name": "baz", "role": "you"}])
+    assert result == EXIT_OK
+    assert MOCK_OUT == ["Task diffs found in instance 0",
+        "\tField 'owner.role' is 'you' local, but 'me' remote",
+        "1 total diff(s) found"]
+
+  def test_success_with_diffs_one_exclude_owner_role(self):
+    # owner.role different in task 0
+    result = self._test_successful_diff_generic([{"name": "foo", "role": "me"},
+            {"name": "bar", "role": "you"}, {"name": "baz", "role": "you"}],
+        [{"name": "foo", "role": "you"}, {"name": "bar", "role": "you"},
+            {"name": "baz", "role": "you"}],
+        excludes=["owner.role"])
+    assert result == EXIT_OK
+    assert MOCK_OUT == ["No diffs found!"]
+
+  def test_success_with_diffs_one_json(self):
+    # owner.role different in task 0
+    result = self._test_successful_diff_generic([{"name": "foo", "role": "me"},
+            {"name": "bar", "role": "you"}, {"name": "baz", "role": "you"}],
+        [{"name": "foo", "role": "you"}, {"name": "bar", "role": "you"},
+            {"name": "baz", "role": "you"}], write_json=True)
+    assert result == EXIT_OK
+    out_json = json.loads(''.join(MOCK_OUT))
+    assert len(out_json) == 1
+    assert out_json[0]["task"] == 0
+    assert out_json[0]["difftype"] == "fields"
+    assert len(out_json[0]["fields"]) == 1
+    assert out_json[0]["fields"][0]["field"] == "owner.role"
+    assert out_json[0]["fields"][0]["local"] == "you"
+    assert out_json[0]["fields"][0]["remote"] == "me"
+
+  def test_success_with_diffs_two(self):
+    # local has more tasks than remote
+    result = self._test_successful_diff_generic([{"name": "foo", "role": "you"},
+            {"name": "bar", "role": "you"}],
+        [{"name": "foo", "role": "you"}, {"name": "bar", "role": "you"},
+            {"name": "baz", "role": "you"}])
+    assert result == EXIT_OK
+    assert MOCK_OUT == ["Local config has a different number of tasks: 3 local vs 2 running",
+        "1 total diff(s) found"]
+
+  def test_success_with_diffs_two_and_a_half(self):
+    # Reverse of test two
+    result = self._test_successful_diff_generic([{"name": "foo", "role": "you"},
+            {"name": "bar", "role": "you"}, {"name": "baz", "role": "you"}],
+        [{"name": "foo", "role": "you"}, {"name": "bar", "role": "you"}])
+    assert result == EXIT_OK
+    assert MOCK_OUT == ["Local config has a different number of tasks: 2 local vs 3 running",
+        "1 total diff(s) found"]
+
+  def test_success_with_diffs_two_json(self):
+    # local has more tasks than remote
+    result = self._test_successful_diff_generic([{"name": "foo", "role": "you"},
+            {"name": "bar", "role": "you"}],
+        [{"name": "foo", "role": "you"}, {"name": "bar", "role": "you"},
+            {"name": "baz", "role": "you"}], write_json=True)
+    assert result == EXIT_OK
+    out_json = json.loads("".join(MOCK_OUT))
+    assert len(out_json) == 1
+    assert out_json[0]["difftype"] == "num_tasks"
+    assert out_json[0]["local"] == 3
+    assert out_json[0]["remote"] == 2
+
+  def test_success_with_diffs_three(self):
+    # local has more tasks than remote, and local task 1 has a different numCpus
+    result = self._test_successful_diff_generic([{"name": "foo", "role": "you"},
+            {"name": "bar", "role": "you"}],
+        [{"name": "foo", "role": "you"}, {"name": "bar", "role": "you", "num_cpus": 4},
+            {"name": "baz", "role": "you"}])
+    assert result == EXIT_OK
+    assert MOCK_OUT == ["Local config has a different number of tasks: 3 local vs 2 running",
+        "Task diffs found in instance 1",
+        "\tField 'numCpus' is '4' local, but '2' remote",
+        "2 total diff(s) found"]
+
+  def test_success_with_diffs_three_json(self):
+    # local has more tasks than remote, and local task 1 has a different numCpus
+    result = self._test_successful_diff_generic([{"name": "foo", "role": "you"},
+            {"name": "bar", "role": "you"}],
+        [{"name": "foo", "role": "you"}, {"name": "bar", "role": "you", "num_cpus": 4},
+            {"name": "baz", "role": "you"}], write_json=True)
+    assert result == EXIT_OK
+    json_out = json.loads("".join(MOCK_OUT))
+    assert len(json_out) == 2
+    def matches_numtasks_diff(f):
+      return f["difftype"] == "num_tasks" and f["remote"] == 2 and f["local"] == 3
+    assert any(matches_numtasks_diff(entry) for entry in json_out)
+
+    def matches_fields_diff(f):
+      if f["difftype"] != "fields" or f["task"] != 1:
+        return False
+      if len(f["fields"]) != 1:
+        return False
+      field = f["fields"][0]
+      return field["field"] == "numCpus" and field["local"] == 4 and field["remote"] == 2
+    assert any(matches_fields_diff(entry) for entry in json_out)
+
+  def test_success_with_diffs_four(self):
+    # Same number of tasks, but task 0 has a different name, task 1 has a different role,
+    # and task 3 has a different number of cpus.
+    result = self._test_successful_diff_generic([{"name": "foobie", "role": "you"},
+            {"name": "bar", "role": "him"}, {"name": "baz", "role": "you", "num_cpus": 3}],
+        [{"name": "foo", "role": "you"}, {"name": "bar", "role": "you"},
+         {"name": "baz", "role": "you", "num_cpus": 4}])
+
+    assert result == EXIT_OK
+    assert MOCK_OUT == ["Task diffs found in instance 0",
+       "\tField 'jobName' is 'foo' local, but 'foobie' remote",
+       "Task diffs found in instance 1",
+       "\tField 'owner.role' is 'you' local, but 'him' remote",
+       "Task diffs found in instance 2",
+       "\tField 'numCpus' is '4' local, but '3' remote",
+       "3 total diff(s) found"]
+
+  def test_success_with_diffs_four_exclude(self):
+    # Same number of tasks, but task 0 has a different name, task 1 has a different role,
+    # and task 3 has a different number of cpus.
+    result = self._test_successful_diff_generic([{"name": "foobie", "role": "you"},
+            {"name": "bar", "role": "him"}, {"name": "baz", "role": "you", "num_cpus": 3}],
+        [{"name": "foo", "role": "you"}, {"name": "bar", "role": "you"},
+         {"name": "baz", "role": "you", "num_cpus": 4}],
+        excludes=["jobName", "owner.role"])
+
+    assert result == EXIT_OK
+    assert MOCK_OUT == ["Task diffs found in instance 2",
+       "\tField 'numCpus' is '4' local, but '3' remote",
+       "1 total diff(s) found"]
+
+  def _test_successful_diff_generic(
+      self,
+      remote_task_spec,
+      local_task_spec,
+      write_json=False,
+      excludes=None):
+
+    """Generic version of json-tree diff test.
+    :param remote_task_spec: a list of dictionaries, containing parameters used to fill in
+        the task configs generated by the test for mock calls to getTaskStatus.
+    :param local_task_spec: a list of dictionaries, containing parameters used to fill in
+        the task configs generated by the test for mock calls to populateJobConfig.
+    :param write_json: flag indicating whether the test should generate json output or
+        user-friendly output.
+    :param excludes: a list of fields that should be specified for exclusion in the test
+       using --exclude-field parameters.
+
+    For the task_spec parameters, the dictionaries can contain the following keys:
+    - name: mandatory field containing the job name for the task.
+    - role: the value of the "role" field.
+    - num_cpus: the value of the numCpus field
+    - max_task_failures: the value of the maxTaskFailures field.
+    """
+    clear_mock_log()
+    clear_mock_out()
     (mock_api, mock_scheduler_proxy) = self.create_mock_api()
+    def foo(*args, **kwargs):
+      return mock_scheduler_proxy
+
     with contextlib.nested(
         patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
         patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS),
-        patch('subprocess.call', return_value=0),
-        patch('json.loads', return_value=Mock())) as (_, _, subprocess_patch, _):
-      mock_scheduler_proxy.getTasksStatus.return_value = self.create_status_response()
-      self.setup_populate_job_config(mock_scheduler_proxy)
+        patch('apache.aurora.client.cli.print_aurora_log', side_effect=mock_log),
+        patch('apache.aurora.client.cli.context.AuroraCommandContext.print_out',
+            side_effect=mock_out),
+        patch('subprocess.call', return_value=0)):
+      mock_scheduler_proxy.getTasksStatus.return_value = self.create_status_response(
+          remote_task_spec)
+      self.setup_populate_job_config(mock_scheduler_proxy, local_task_spec)
       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])
+        params = ['job', 'diff', 'west/bozo/test/hello', fp.name]
+        if write_json:
+          params.append("--write-json")
+        if excludes is not None:
+          for e in excludes:
+            params.append("--exclude-field=%s" % e)
+        result = cmd.execute(params)
 
         # Diff should get the task status, populate a config, and run diff.
         mock_scheduler_proxy.getTasksStatus.assert_called_with(
@@ -127,10 +413,8 @@ class TestDiffCommand(AuroraClientCommandTest):
         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 with two parameters.
-        assert subprocess_patch.call_count == 1
-        assert len(subprocess_patch.call_args[0][0]) == 3
-        assert subprocess_patch.call_args[0][0][0] == os.environ.get('DIFF_VIEWER', 'diff')
+        return result
+
 
   def test_diff_invalid_config(self):
     """Test the diff command if the user passes a config with an error in it."""
@@ -141,12 +425,10 @@ class TestDiffCommand(AuroraClientCommandTest):
     with contextlib.nested(
         patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
         patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS),
-        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,
             mock_clusters,
-            options,
             subprocess_patch,
             json_patch):
       with temporary_file() as fp:
@@ -158,6 +440,7 @@ class TestDiffCommand(AuroraClientCommandTest):
         assert mock_scheduler_proxy.getTasksStatus.call_count == 0
         assert mock_scheduler_proxy.populateJobConfig.call_count == 0
         assert subprocess_patch.call_count == 0
+        return result
 
   def test_diff_server_error(self):
     """Test the diff command if the user passes a config with an error in it."""
@@ -168,12 +451,10 @@ class TestDiffCommand(AuroraClientCommandTest):
     with contextlib.nested(
         patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
         patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS),
-        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,
             mock_clusters,
-            options,
             subprocess_patch,
             json_patch):
       with temporary_file() as fp:
@@ -189,3 +470,32 @@ class TestDiffCommand(AuroraClientCommandTest):
                 statuses=ACTIVE_STATES))
         assert mock_scheduler_proxy.populateJobConfig.call_count == 0
         assert subprocess_patch.call_count == 0
+
+  def test_successful_unix_diff(self):
+    """Test the old shell-based diff method."""
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
+    with contextlib.nested(
+        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
+        patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS),
+        patch('subprocess.call', return_value=0),
+        patch('json.loads', return_value=Mock())) 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', '--use-shell-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(jobName='hello', environment='test', owner=Identity(role='bozo'),
+                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 with two parameters.
+        assert subprocess_patch.call_count == 1
+        assert len(subprocess_patch.call_args[0][0]) == 3
+        assert subprocess_patch.call_args[0][0][0] == os.environ.get('DIFF_VIEWER', 'diff')

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/92799eae/src/test/python/apache/aurora/client/cli/test_json_diff.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/test_json_diff.py b/src/test/python/apache/aurora/client/cli/test_json_diff.py
new file mode 100644
index 0000000..8754089
--- /dev/null
+++ b/src/test/python/apache/aurora/client/cli/test_json_diff.py
@@ -0,0 +1,100 @@
+#
+# 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.
+#
+
+import unittest
+
+from apache.aurora.client.cli.json_tree_diff import (
+    compare_json,
+    compare_pruned_json,
+    FieldDifference,
+    prune_structure
+)
+
+
+class TestJsonDiff(unittest.TestCase):
+
+  LIST_ONE = {
+      "a": [1, 2, 3],
+      "x": {
+          "y": {
+              "z": 3,
+              "a": {
+                  "c": 2,
+                  "d": 3
+              }
+          },
+          "z": {
+              "a": 27,
+              "b": "foo",
+          },
+          "q": "br",
+      },
+      "y": "foo",
+      "z": "zoom",
+      "q": "fizzboom"
+  }
+
+  LIST_TWO = {
+      "a": [3, 2, 1],
+      "x": {
+          "y": {
+              "z": 5,
+              "a": {
+                  "c": 2,
+                  "d": 3
+              }
+          },
+          "z": {
+              "a": 27,
+              "b": "foo"
+          },
+          "q": "bar",
+      },
+      "y": "foo",
+      "z": "zoom",
+  }
+
+  def test_pruner(self):
+    ex = [["x", "y", "z"], ["x", "y", "a"], ["x", "z"], "q"]
+    assert prune_structure(self.LIST_ONE, ex) == {
+        "a": [1, 2, 3],
+        "x": {
+            "y": {},
+            "q": "br",
+        },
+        "y": "foo",
+        "z": "zoom"
+    }
+
+  def test_compare_canonicalized(self):
+    one = {"a": ["1", "2", ["3", "4"]]}
+    two = {"a": ["2", ["3", "4"], "1"]}
+    assert compare_json(one, two, []) == []
+
+  def test_compare_json(self):
+    result = compare_json(self.LIST_ONE, self.LIST_TWO, [])
+    expected = [FieldDifference(name='q', base='fizzboom', other='__undefined__'),
+        FieldDifference(name='x.q', base='br', other='bar'),
+        FieldDifference(name='x.y.z', base=3, other=5)]
+    assert result == expected
+
+  def test_compare_pruned(self):
+    assert compare_pruned_json(self.LIST_ONE, self.LIST_TWO, [['x', 'y']]) == [
+        FieldDifference(name='q', base='fizzboom', other='__undefined__'),
+        FieldDifference(name='x.q', base='br', other='bar')]
+
+    assert compare_pruned_json(self.LIST_ONE, self.LIST_TWO, [['x', 'y'], ['x', 'q']]) == [
+        FieldDifference(name='q', base='fizzboom', other='__undefined__')]
+
+    assert compare_pruned_json(self.LIST_ONE, self.LIST_TWO, ['x', 'q']) == []