Return-Path: X-Original-To: apmail-aurora-commits-archive@minotaur.apache.org Delivered-To: apmail-aurora-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 693FB1156B for ; Tue, 9 Sep 2014 14:32:57 +0000 (UTC) Received: (qmail 88610 invoked by uid 500); 9 Sep 2014 14:32:57 -0000 Delivered-To: apmail-aurora-commits-archive@aurora.apache.org Received: (qmail 88575 invoked by uid 500); 9 Sep 2014 14:32:57 -0000 Mailing-List: contact commits-help@aurora.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@aurora.incubator.apache.org Delivered-To: mailing list commits@aurora.incubator.apache.org Received: (qmail 88566 invoked by uid 99); 9 Sep 2014 14:32:57 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 09 Sep 2014 14:32:57 +0000 X-ASF-Spam-Status: No, hits=-2001.6 required=5.0 tests=ALL_TRUSTED,RP_MATCHES_RCVD,T_FILL_THIS_FORM_FRAUD_PHISH,T_FILL_THIS_FORM_SHORT X-Spam-Check-By: apache.org Received: from [140.211.11.3] (HELO mail.apache.org) (140.211.11.3) by apache.org (qpsmtpd/0.29) with SMTP; Tue, 09 Sep 2014 14:32:29 +0000 Received: (qmail 87985 invoked by uid 99); 9 Sep 2014 14:32:27 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 09 Sep 2014 14:32:27 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id DAC91A10218; Tue, 9 Sep 2014 14:32:26 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: mchucarroll@apache.org To: commits@aurora.incubator.apache.org Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: git commit: Revert "Improve aurora "job diff" command." Date: Tue, 9 Sep 2014 14:32:26 +0000 (UTC) X-Virus-Checked: Checked by ClamAV on apache.org Repository: incubator-aurora Updated Branches: refs/heads/master d30e8ae4c -> 9f594f1de Revert "Improve aurora "job diff" command." This reverts commit d30e8ae4ce7d4883b842cd242570b597eceb2a8d. Project: http://git-wip-us.apache.org/repos/asf/incubator-aurora/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-aurora/commit/9f594f1d Tree: http://git-wip-us.apache.org/repos/asf/incubator-aurora/tree/9f594f1d Diff: http://git-wip-us.apache.org/repos/asf/incubator-aurora/diff/9f594f1d Branch: refs/heads/master Commit: 9f594f1de6bbf46c74863dd3fc4d2708b7a974f2 Parents: d30e8ae Author: Mark Chu-Carroll Authored: Tue Sep 9 10:31:53 2014 -0400 Committer: Mark Chu-Carroll Committed: Tue Sep 9 10:31:53 2014 -0400 ---------------------------------------------------------------------- src/main/python/apache/aurora/client/cli/BUILD | 1 - .../python/apache/aurora/client/cli/jobs.py | 132 +----- .../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 | 424 +++---------------- .../apache/aurora/client/cli/test_json_diff.py | 100 ----- 6 files changed, 66 insertions(+), 714 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/9f594f1d/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 1ffa603..ebe681a 100644 --- a/src/main/python/apache/aurora/client/cli/BUILD +++ b/src/main/python/apache/aurora/client/cli/BUILD @@ -54,7 +54,6 @@ 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/9f594f1d/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 2aa3092..8f349c0 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,7 +38,6 @@ from apache.aurora.client.cli import ( Verb ) from apache.aurora.client.cli.context import AuroraCommandContext -from apache.aurora.client.cli.json_tree_diff import canonicalize_json, compare_pruned_json from apache.aurora.client.cli.options import ( ALL_INSTANCES, BATCH_OPTION, @@ -143,20 +142,8 @@ class DiffCommand(Verb): @property def help(self): return """Compare a job configuration against a running job. - -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} -""" +By default the diff will be displayed using 'diff', though you may choose an +alternate diff program by setting the DIFF_VIEWER environment variable.""" @property def name(self): @@ -164,26 +151,10 @@ where each field_diff is: {"field": fieldname, "local": value, "remote": value} 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 = canonicalize_json(serialize(task, - protocol_factory=TJSONProtocol.TSimpleJSONProtocolFactory())) - - return json.loads(data) - def pretty_print_task(self, task): task.configuration = None task.executorConfig = ExecutorConfig( @@ -199,84 +170,6 @@ where each field_diff is: {"field": fieldname, "local": value, "remote": value} 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: @@ -289,22 +182,27 @@ where each field_diff is: {"field": fieldname, "local": value, "remote": value} 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 - 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) - + 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 class InspectCommand(Verb): http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/9f594f1d/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 deleted file mode 100644 index 82a78f4..0000000 --- a/src/main/python/apache/aurora/client/cli/json_tree_diff.py +++ /dev/null @@ -1,116 +0,0 @@ -# -# 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"]) - - -# 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_json(val): - def canonicalize_list(lst): - result = [] - for l in lst: - result.append(canonicalize_json(l)) - result.sort() - return tuple(result) - - def canonicalize_dict(dct): - result = {} - for key in dct: - result[key] = canonicalize_json(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 - - -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.)""" - - keys = set(base.keys()) | set(other.keys()) - differences = [] - for key in keys: - base_val = canonicalize_json(base.get(key, "__undefined__")) - other_val = canonicalize_json(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/9f594f1d/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 43e6739..e1f9ebf 100644 --- a/src/test/python/apache/aurora/client/cli/BUILD +++ b/src/test/python/apache/aurora/client/cli/BUILD @@ -22,7 +22,6 @@ python_test_suite( pants(':help'), pants(':inspect'), pants(':job'), - pants(':json_diff'), pants(':config'), pants(':logging'), pants(':plugins'), @@ -32,12 +31,6 @@ 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/9f594f1d/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 7fefc16..38629b6 100644 --- a/src/test/python/apache/aurora/client/cli/test_diff.py +++ b/src/test/python/apache/aurora/client/cli/test_diff.py @@ -13,13 +13,12 @@ # 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, EXIT_OK +from apache.aurora.client.cli import EXIT_INVALID_CONFIGURATION, EXIT_INVALID_PARAMETER from apache.aurora.client.cli.client import AuroraCommandLine from apache.aurora.client.cli.util import AuroraClientCommandTest @@ -30,12 +29,8 @@ from gen.apache.aurora.api.ttypes import ( Identity, JobConfiguration, JobKey, - Metadata, PopulateJobResult, - Response, ResponseCode, - Result, - ScheduledTask, ScheduleStatus, ScheduleStatusResult, TaskConfig, @@ -43,41 +38,8 @@ 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): - - def setUp(self): - clear_mock_out() - clear_mock_log() - - def tearDown(self): - clear_mock_out() - clear_mock_log() - - @classmethod def setup_mock_options(cls): """set up to get a mock options object.""" @@ -92,72 +54,41 @@ class TestDiffCommand(AuroraClientCommandTest): return mock_options @classmethod - 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 + 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 @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, specs=None): + def create_status_response(cls): resp = cls.create_simple_success_response() resp.result.scheduleStatusResult = Mock(spec=ScheduleStatusResult) - resp.result.scheduleStatusResult.tasks = set(cls.create_mock_scheduled_tasks(specs)) + resp.result.scheduleStatusResult.tasks = set(cls.create_mock_scheduled_tasks()) return resp @classmethod @@ -165,257 +96,28 @@ class TestDiffCommand(AuroraClientCommandTest): return cls.create_blank_response(ResponseCode.INVALID_REQUEST, 'No tasks found for query') @classmethod - 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) + def setup_populate_job_config(cls, api): + populate = cls.create_simple_success_response() + populate.result.populateJobResult = Mock(spec=PopulateJobResult) api.populateJobConfig.return_value = populate + populate.result.populateJobResult.populated = cls.create_mock_scheduled_tasks() return populate - 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="ix0", value="2")]}, - {"name": "serv", "metadata": [Metadata(key="a", value="1"), - Metadata(key="b", value="2"), Metadata(key="ix0", value="2")]}, - {"name": "serv", "metadata": [Metadata(key="a", value="1"), - Metadata(key="b", value="2"), Metadata(key="ix0", value="2")]}] - - two = [{"name": "serv", "metadata": [Metadata(key="b", value="2"), - Metadata(key="a", value="1"), Metadata(key="ix0", value="2")]}, - {"name": "serv", "metadata": [Metadata(key="ix0", value="2"), - Metadata(key="a", value="1"), Metadata(key="b", value="2")]}, - {"name": "serv", "metadata": [Metadata(key="a", value="1"), - Metadata(key="ix0", 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="ix0", value="2")]}, - {"name": "serv", "metadata": [Metadata(key="a", value="1"), - Metadata(key="b", value="2"), Metadata(key="ix0", value="2")]}, - {"name": "serv", "metadata": [Metadata(key="a", value="1"), - Metadata(key="b", value="2"), Metadata(key="ix0", value="2")]}] - - two = [{"name": "serv", "metadata": [Metadata(key="b", value="2"), - Metadata(key="a", value="1"), Metadata(key="ix0", value="2")]}, - {"name": "serv", "metadata": [Metadata(key="ix0", value="2"), - Metadata(key="a", value="3"), Metadata(key="b", value="2")]}, - {"name": "serv", "metadata": [Metadata(key="a", value="1"), - Metadata(key="ix0", 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', - ("\tField 'metadata' is " - "'({u'key': u'a', u'value': u'3'}, {u'key': u'b', u'value': u'2'}," - " {u'key': u'ix0', u'value': u'2'})' local, but " - "'({u'key': u'a', u'value': u'1'}, {u'key': u'b', u'value': u'2'}, " - "{u'key': u'ix0', u'value': u'2'})' 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() + def test_successful_diff(self): + """Test the diff command.""" (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('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) + 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() - 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) + 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( @@ -425,8 +127,10 @@ 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')) - return result - + # 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') def test_diff_invalid_config(self): """Test the diff command if the user passes a config with an error in it.""" @@ -437,10 +141,12 @@ 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: @@ -452,7 +158,6 @@ 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.""" @@ -463,10 +168,12 @@ 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: @@ -482,32 +189,3 @@ 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/9f594f1d/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 deleted file mode 100644 index 8754089..0000000 --- a/src/test/python/apache/aurora/client/cli/test_json_diff.py +++ /dev/null @@ -1,100 +0,0 @@ -# -# 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']) == []