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 8EB3E1033A for ; Mon, 16 Dec 2013 18:14:25 +0000 (UTC) Received: (qmail 95115 invoked by uid 500); 16 Dec 2013 18:14:25 -0000 Delivered-To: apmail-aurora-commits-archive@aurora.apache.org Received: (qmail 95091 invoked by uid 500); 16 Dec 2013 18:14:25 -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 95084 invoked by uid 99); 16 Dec 2013 18:14:25 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 16 Dec 2013 18:14:25 +0000 X-ASF-Spam-Status: No, hits=-2000.5 required=5.0 tests=ALL_TRUSTED,RP_MATCHES_RCVD,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; Mon, 16 Dec 2013 18:14:22 +0000 Received: (qmail 94933 invoked by uid 99); 16 Dec 2013 18:14:02 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 16 Dec 2013 18:14:02 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 1917890DCE7; Mon, 16 Dec 2013 18:14:02 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: wickman@apache.org To: commits@aurora.incubator.apache.org Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: git commit: First steps towards aurora client v2 Date: Mon, 16 Dec 2013 18:14:02 +0000 (UTC) X-Virus-Checked: Checked by ClamAV on apache.org Updated Branches: refs/heads/master a51cd19ee -> 39a94c1a9 First steps towards aurora client v2 First step towards aurora client v2! - Initial implementation of the noun/verb command and options processing framework. - Initial implementation of a command processing context for aurora commands. - Implementations of a "Job" noun and "create" and "kill" verbs. - Tests. Note: not all of the v1 tests for the create and kill verbs have been migrated to the new framework; command processing contexts need a bit more work to make it easy to do the appropriate mocking/stubbing to support them. They'll be in the next change. Reviewed at https://reviews.apache.org/r/16130/ Project: http://git-wip-us.apache.org/repos/asf/incubator-aurora/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-aurora/commit/39a94c1a Tree: http://git-wip-us.apache.org/repos/asf/incubator-aurora/tree/39a94c1a Diff: http://git-wip-us.apache.org/repos/asf/incubator-aurora/diff/39a94c1a Branch: refs/heads/master Commit: 39a94c1a95051eb88c2c9ca6ac6da124032ca14a Parents: a51cd19 Author: Mark Chu-Carroll Authored: Mon Dec 16 10:09:56 2013 -0800 Committer: Brian Wickman Committed: Mon Dec 16 10:09:56 2013 -0800 ---------------------------------------------------------------------- src/main/python/twitter/aurora/BUILD.thirdparty | 1 + src/main/python/twitter/aurora/client/cli/BUILD | 26 +++ .../twitter/aurora/client/cli/__init__.py | 197 ++++++++++++++++++ .../python/twitter/aurora/client/cli/context.py | 49 +++++ .../python/twitter/aurora/client/cli/jobs.py | 114 +++++++++++ src/test/python/twitter/aurora/client/cli/BUILD | 26 +++ .../twitter/aurora/client/cli/test_create.py | 200 +++++++++++++++++++ .../twitter/aurora/client/cli/test_kill.py | 114 +++++++++++ .../python/twitter/aurora/client/cli/util.py | 152 ++++++++++++++ .../twitter/aurora/client/commands/test_kill.py | 2 +- 10 files changed, 880 insertions(+), 1 deletion(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/39a94c1a/src/main/python/twitter/aurora/BUILD.thirdparty ---------------------------------------------------------------------- diff --git a/src/main/python/twitter/aurora/BUILD.thirdparty b/src/main/python/twitter/aurora/BUILD.thirdparty index 87156ea..dd0d4bc 100644 --- a/src/main/python/twitter/aurora/BUILD.thirdparty +++ b/src/main/python/twitter/aurora/BUILD.thirdparty @@ -9,6 +9,7 @@ def make_dep(name, version, dependency_name=None): versioned_name = "%s==%s" % (dependency_name, version) python_requirement(requirement=versioned_name, name=name) +make_dep('argparse', '1.2.1') make_dep('mesos-core', '0.15.0-rc4', 'mesos') make_dep('mock', '1.0.1') make_dep('mox', '0.5.3') http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/39a94c1a/src/main/python/twitter/aurora/client/cli/BUILD ---------------------------------------------------------------------- diff --git a/src/main/python/twitter/aurora/client/cli/BUILD b/src/main/python/twitter/aurora/client/cli/BUILD new file mode 100644 index 0000000..aaeede4 --- /dev/null +++ b/src/main/python/twitter/aurora/client/cli/BUILD @@ -0,0 +1,26 @@ + +python_binary( + name='client', + entry_point = 'twitter.aurora.cli:main', + dependencies = [ pants(':cli') ], + ) + +python_library( + name='cli', + sources = [ '__init__.py', 'context.py', 'jobs.py' ], + dependencies = [ + pants('aurora/twitterdeps/src/python/twitter/common/python'), + pants('src/main/python/twitter/aurora/client/api:command_runner'), + pants('src/main/python/twitter/aurora/client/api:disambiguator'), + pants('src/main/python/twitter/aurora/client/api:job_monitor'), + pants('src/main/python/twitter/aurora/client/api:updater'), + pants('src/main/python/twitter/aurora/client/hooks'), + pants('src/main/python/twitter/aurora/client:base'), + pants('src/main/python/twitter/aurora/client:config'), + pants('src/main/python/twitter/aurora/client:factory'), + pants('src/main/python/twitter/aurora/client:options'), + pants('src/main/python/twitter/aurora/common'), + pants('src/main/thrift/com/twitter/aurora/gen:py-thrift'), + pants('src/main/python/twitter/aurora:argparse') + ] + ) http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/39a94c1a/src/main/python/twitter/aurora/client/cli/__init__.py ---------------------------------------------------------------------- diff --git a/src/main/python/twitter/aurora/client/cli/__init__.py b/src/main/python/twitter/aurora/client/cli/__init__.py new file mode 100644 index 0000000..4b771d6 --- /dev/null +++ b/src/main/python/twitter/aurora/client/cli/__init__.py @@ -0,0 +1,197 @@ +'''Command-line tooling infrastructure for aurora client v2. + +This provides a framework for a noun/verb command-line application. The application is structured +around a collection of basic objects (nouns) that can be manipulated by the command line, where +each type of object provides a collection of operations (verbs). Every command invocation +consists of the name of the noun, followed by one of the verbs for that noun, followed by other +arguments needed by the verb. + +For example: +- To create a job, the noun is "job", the verb is "create": + $ aurora job create us-west/www/prod/server server.aurora + +- To find out the resource quota for a specific user, the noun is "user" and the verb is + "get_quota": + $ aurora user get_quota mchucarroll +''' + +from __future__ import print_function + +import sys + +from abc import abstractmethod + +import argparse + + +# Constants for standard return codes. +EXIT_OK = 0 +EXIT_INVALID_CONFIGURATION = 3 +EXIT_COMMAND_FAILURE = 4 +EXIT_INVALID_COMMAND = 5 +EXIT_INVALID_PARAMETER = 6 +EXIT_NETWORK_ERROR = 7 +EXIT_PERMISSION_VIOLATION = 8 +EXIT_TIMEOUT = 9 +EXIT_UNKNOWN_ERROR = 20 + + +class Context(object): + class Error(Exception): pass + + class ArgumentException(Error): pass + + class CommandError(Error): + def __init__(self, code, msg): + super(Context.CommandError, self).__init__(msg) + self.msg = msg + self.code = code + + def set_options(self, options): + """Add the options object to a context. + This is separated from the constructor to make patching tests easier. + """ + self.options = options + + +class AuroraCommand(object): + def setup_options_parser(self, argparser): + """Set up command line options parsing for this command. + This is a thin veneer over the standard python argparse system. + :param argparser: the argument parser where this command can add its arguments. + """ + pass + + @property + def help(self): + """The help message for a command that will be used in the argparse help message""" + + @property + def name(self): + """The command name""" + + +class CommandLine(object): + """The top-level object implementing a command-line application.""" + + def __init__(self): + self.nouns = {} + self.parser = None + + def register_noun(self, noun): + """Add a noun to the application""" + if not isinstance(noun, Noun): + raise TypeError('register_noun requires a Noun argument') + self.nouns[noun.name] = noun + + def setup_options_parser(self): + """ Build the options parsing for the application.""" + self.parser = argparse.ArgumentParser() + subparser = self.parser.add_subparsers(dest='noun') + for (name, noun) in self.nouns.items(): + noun_parser = subparser.add_parser(name, help=noun.help) + noun.internal_setup_options_parser(noun_parser) + + def register_nouns(self): + """This method should overridden by applications to register the collection of nouns + that they can manipulate. + """ + pass + + def execute(self, args): + """Execute a command. + :param args: the command-line arguments for the command. This only includes arguments + that should be parsed by the application; it does not include sys.argv[0]. + """ + self.register_nouns() + self.setup_options_parser() + options = self.parser.parse_args(args) + if options.noun not in self.nouns: + raise ValueError('Unknown command: %s' % options.noun) + noun = self.nouns[options.noun] + context = noun.create_context() + context.set_options(options) + try: + return noun.execute(context) + except Context.CommandError as c: + print('Error executing command: %s' % c.msg, file=sys.stderr) + return c.code + + +class Noun(AuroraCommand): + """A type of object manipulated by a command line application""" + class InvalidVerbException(Exception): pass + + def __init__(self): + super(Noun, self).__init__() + self.verbs = {} + + def register_verb(self, verb): + """Add an operation supported for this noun.""" + if not isinstance(verb, Verb): + raise TypeError('register_verb requires a Verb argument') + self.verbs[verb.name] = verb + verb._register(self) + + def internal_setup_options_parser(self, argparser): + """Internal driver for the options processing framework.""" + self.setup_options_parser(argparser) + subparser = argparser.add_subparsers(dest='verb') + for (name, verb) in self.verbs.items(): + vparser = subparser.add_parser(name, help=verb.help) + verb.setup_options_parser(vparser) + + @classmethod + def create_context(cls): + """Commands access state through a context object. The noun specifies what kind + of context should be created for this noun's required state. + """ + pass + + @abstractmethod + def setup_options_parser(self, argparser): + pass + + def execute(self, context): + if context.options.verb not in self.verbs: + raise self.InvalidVerbException('Noun %s does not have a verb %s' % + (self.name, context.options.verb)) + self.verbs[context.options.verb].execute(context) + + +class Verb(AuroraCommand): + """An operation for a noun. Most application logic will live in verbs.""" + + def _register(self, noun): + """Create a link from a verb to its noun.""" + self.noun = noun + + @abstractmethod + def setup_options_parser(self, argparser): + pass + + def execute(self, context): + pass + + +class AuroraCommandLine(CommandLine): + """ An example implementation of a command line application using this framework. + This should probably eventually get moved in to its own source file. + """ + + @classmethod + def get_description(cls): + return 'Aurora client command line' + + def register_nouns(self): + from .jobs import Job + self.register_noun(Job()) + + +def main(): + cmd = AuroraCommandLine() + cmd.execute(sys.argv[1:]) + + +if __name__ == '__main__': + main(sys.argv) http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/39a94c1a/src/main/python/twitter/aurora/client/cli/context.py ---------------------------------------------------------------------- diff --git a/src/main/python/twitter/aurora/client/cli/context.py b/src/main/python/twitter/aurora/client/cli/context.py new file mode 100644 index 0000000..2ae92ec --- /dev/null +++ b/src/main/python/twitter/aurora/client/cli/context.py @@ -0,0 +1,49 @@ + +from twitter.aurora.client.base import synthesize_url +from twitter.aurora.client.cli import Context, EXIT_NETWORK_ERROR +from twitter.aurora.client.config import get_config +from twitter.aurora.client.factory import make_client +from twitter.common import log + +from gen.twitter.aurora.ttypes import ResponseCode + + +class AuroraCommandContext(Context): + """A context object used by Aurora commands to manage command processing state + and common operations. + """ + + def get_api(self, cluster): + """Creates an API object for a specified cluster""" + return make_client(cluster) + + def get_job_config(self, job_key, config_file): + """Loads a job configuration from a config file""" + jobname = job_key.name + return get_config( + jobname, + config_file, + self.options.json, + self.options.bindings, + select_cluster=job_key.cluster, + select_role=job_key.role, + select_env=job_key.env) + + def open_page(self, url): + import webbrowser + webbrowser.open_new_tab(url) + + def open_job_page(self, api, config): + self.open_page(synthesize_url(api.scheduler.scheduler().url, config.role(), + config.environment(), config.name())) + + def handle_open(self, api): + if self.options.open_browser: + self.open_page(synthesize_url(api.scheduler.scheduler().url, + self.options.jobspec.role, self.options.jobspec.env, self.options.jobspec.name)) + + def check_and_log_response(self, resp): + log.info('Response from scheduler: %s (message: %s)' + % (ResponseCode._VALUES_TO_NAMES[resp.responseCode], resp.message)) + if resp.responseCode != ResponseCode.OK: + raise self.CommandError(EXIT_NETWORK_ERROR, resp.message) http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/39a94c1a/src/main/python/twitter/aurora/client/cli/jobs.py ---------------------------------------------------------------------- diff --git a/src/main/python/twitter/aurora/client/cli/jobs.py b/src/main/python/twitter/aurora/client/cli/jobs.py new file mode 100644 index 0000000..58a723f --- /dev/null +++ b/src/main/python/twitter/aurora/client/cli/jobs.py @@ -0,0 +1,114 @@ +from twitter.aurora.client.api.job_monitor import JobMonitor +from twitter.aurora.client.cli import ( + EXIT_INVALID_CONFIGURATION, + Noun, + Verb +) +from twitter.aurora.client.cli.context import AuroraCommandContext +from twitter.aurora.common.aurora_job_key import AuroraJobKey + +from pystachio.config import Config + + +def parse_instances(instances): + """Parse lists of instances or instance ranges into a set(). + + Examples: + 0-2 + 0,1-3,5 + 1,3,5 + """ + if instances is None or instances == '': + return None + result = set() + for part in instances.split(','): + x = part.split('-') + result.update(range(int(x[0]), int(x[-1]) + 1)) + return sorted(result) + + +class CreateJobCommand(Verb): + @property + def name(self): + return 'create' + + @property + def help(self): + return 'Create a job using aurora' + + CREATE_STATES = ('PENDING', 'RUNNING', 'FINISHED') + + def setup_options_parser(self, parser): + parser.add_argument('--bind', type=str, default=[], dest='bindings', + action='append', + help='Bind a thermos mustache variable name to a value. ' + 'Multiple flags may be used to specify multiple values.') + parser.add_argument('--open-browser', default=False, dest='open_browser', action='store_true', + help='open browser to view job page after job is created') + parser.add_argument('--json', default=False, dest='json', action='store_true', + help='Read job configuration in json format') + parser.add_argument('--wait_until', choices=self.CREATE_STATES, + default='PENDING', + help=('Block the client until all the tasks have transitioned into the requested state. ' + 'Default: PENDING')) + parser.add_argument('jobspec', type=AuroraJobKey.from_path) + parser.add_argument('config_file', type=str) + + def execute(self, context): + try: + config = context.get_job_config(context.options.jobspec, context.options.config_file) + except Config.InvalidConfigError as e: + raise context.CommandError(EXIT_INVALID_CONFIGURATION, + 'Error loading job configuration: %s' % e) + api = context.get_api(config.cluster()) + monitor = JobMonitor(api, config.role(), config.environment(), config.name()) + resp = api.create_job(config) + context.check_and_log_response(resp) + if context.options.open_browser: + context.open_job_page(api, config) + if context.options.wait_until == 'RUNNING': + monitor.wait_until(monitor.running_or_finished) + elif context.options.wait_until == 'FINISHED': + monitor.wait_until(monitor.terminal) + + +class KillJobCommand(Verb): + @property + def name(self): + return 'kill' + + def setup_options_parser(self, parser): + parser.add_argument('--open-browser', default=False, dest='open_browser', action='store_true', + help='open browser to view job page after job is created') + parser.add_argument('--instances', type=parse_instances, dest='instances', default=None, + help='A list of instance ids to act on. Can either be a comma-separated list (e.g. 0,1,2) ' + 'or a range (e.g. 0-2) or any combination of the two (e.g. 0-2,5,7-9). If not set, ' + 'all instances will be acted on.') + parser.add_argument('--config', type=str, default=None, dest='config', + help='Config file for the job, possibly containing hooks') + parser.add_argument('jobspec', type=AuroraJobKey.from_path) + + def execute(self, context): + api = context.get_api(context.options.jobspec.cluster) + resp = api.kill_job(context.options.jobspec, context.options.instances) + context.check_and_log_response(resp) + context.handle_open(api) + + +class Job(Noun): + @property + def name(self): + return 'job' + + @property + def help(self): + return "Work with an aurora job" + + @classmethod + def create_context(cls): + return AuroraCommandContext() + + def __init__(self): + super(Job, self).__init__() + self.register_verb(CreateJobCommand()) + self.register_verb(KillJobCommand()) http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/39a94c1a/src/test/python/twitter/aurora/client/cli/BUILD ---------------------------------------------------------------------- diff --git a/src/test/python/twitter/aurora/client/cli/BUILD b/src/test/python/twitter/aurora/client/cli/BUILD new file mode 100644 index 0000000..89184b6 --- /dev/null +++ b/src/test/python/twitter/aurora/client/cli/BUILD @@ -0,0 +1,26 @@ +python_test_suite( + name = 'all', + dependencies = [ pants(':job') ] +) + +python_library( + name = 'util', + sources = [ 'util.py' ], + dependencies = [ + pants('src/main/python/twitter/aurora/BUILD.thirdparty:mock'), + pants('src/main/python/twitter/aurora/client/cli'), + ] +) + +python_tests( + name = 'job', + sources = [ 'test_create.py', 'test_kill.py' ], + dependencies = [ + pants(':util'), + pants('src/main/python/twitter/aurora/BUILD.thirdparty:mock'), + pants('aurora/twitterdeps/src/python/twitter/common/contextutil'), + pants('src/main/python/twitter/aurora/client/cli'), + pants('src/main/python/twitter/aurora/client/cli'), + pants('src/test/python/twitter/aurora/client/commands:util') + ] +) http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/39a94c1a/src/test/python/twitter/aurora/client/cli/test_create.py ---------------------------------------------------------------------- diff --git a/src/test/python/twitter/aurora/client/cli/test_create.py b/src/test/python/twitter/aurora/client/cli/test_create.py new file mode 100644 index 0000000..da2828f --- /dev/null +++ b/src/test/python/twitter/aurora/client/cli/test_create.py @@ -0,0 +1,200 @@ +import contextlib + +from twitter.aurora.client.cli import ( + AuroraCommandLine, + EXIT_INVALID_CONFIGURATION, + EXIT_NETWORK_ERROR +) +from twitter.aurora.client.cli.util import AuroraClientCommandTest, FakeAuroraCommandContext +from twitter.aurora.client.hooks.hooked_api import HookedAuroraClientAPI +from twitter.aurora.config import AuroraConfig +from twitter.common.contextutil import temporary_file + +from gen.twitter.aurora.ttypes import ( + AssignedTask, + Identity, + ScheduledTask, + ScheduleStatus, + ScheduleStatusResult, + TaskEvent, + TaskQuery, +) + +from mock import Mock, patch + + +class TestClientCreateCommand(AuroraClientCommandTest): + + @classmethod + def setup_mock_options(cls): + """set up to get a mock options object.""" + mock_options = Mock() + mock_options.json = False + mock_options.bindings = {} + mock_options.open_browser = False + mock_options.cluster = None + mock_options.wait_until = 'RUNNING' # or 'FINISHED' for other tests + return mock_options + + @classmethod + def setup_mock_api(cls): + """Builds up a mock API object, with a mock SchedulerProxy""" + mock_api = Mock(spec=HookedAuroraClientAPI) + mock_scheduler = Mock() + mock_scheduler.url = "http://something_or_other" + mock_api.scheduler = mock_scheduler + return (mock_api, mock_scheduler) + + @classmethod + def create_mock_task(cls, task_id, instance_id, initial_time, status): + mock_task = Mock(spec=ScheduledTask) + mock_task.assignedTask = Mock(spec=AssignedTask) + mock_task.assignedTask.taskId = task_id + mock_task.assignedTask.instanceId = instance_id + mock_task.status = status + mock_task_event = Mock(spec=TaskEvent) + mock_task_event.timestamp = initial_time + mock_task.taskEvents = [mock_task_event] + return mock_task + + @classmethod + def create_mock_status_query_result(cls, scheduleStatus): + mock_query_result = cls.create_simple_success_response() + mock_query_result.result.scheduleStatusResult = Mock(spec=ScheduleStatusResult) + if scheduleStatus == ScheduleStatus.INIT: + # status query result for before job is launched. + mock_query_result.result.scheduleStatusResult.tasks = [] + else: + mock_task_one = cls.create_mock_task('hello', 0, 1000, scheduleStatus) + mock_task_two = cls.create_mock_task('hello', 1, 1004, scheduleStatus) + mock_query_result.result.scheduleStatusResult.tasks = [mock_task_one, mock_task_two] + return mock_query_result + + @classmethod + def create_mock_query(cls): + return TaskQuery(owner=Identity(role=cls.TEST_ROLE), environment=cls.TEST_ENV, + jobName=cls.TEST_JOB) + + @classmethod + def get_createjob_response(cls): + # Then, we call api.create_job(config) + return cls.create_simple_success_response() + + @classmethod + def get_failed_createjob_response(cls): + return cls.create_error_response() + + @classmethod + def assert_create_job_called(cls, mock_api): + # Check that create_job was called exactly once, with an AuroraConfig parameter. + assert mock_api.create_job.call_count == 1 + assert isinstance(mock_api.create_job.call_args_list[0][0][0], AuroraConfig) + + @classmethod + def assert_scheduler_called(cls, mock_api, mock_query, num_queries): + print('Calls to getTasksStatus: %s' % mock_api.scheduler.getTasksStatus.call_args_list) + assert mock_api.scheduler.getTasksStatus.call_count == num_queries + mock_api.scheduler.getTasksStatus.assert_called_with(mock_query) + + def test_simple_successful_create_job(self): + """Run a test of the "create" command against a mocked-out API: + Verifies that the creation command sends the right API RPCs, and performs the correct + tests on the result.""" + + # We'll patch out create_context, which will give us a fake context + # object, and everything can be stubbed through that. + mock_context = FakeAuroraCommandContext() + with patch('twitter.aurora.client.cli.jobs.Job.create_context', return_value=mock_context): + # After making the client, create sets up a job monitor. + # The monitor uses TaskQuery to get the tasks. It's called at least twice:once before + # the job is created, and once after. So we need to set up mocks for the query results. + mock_query = self.create_mock_query() + mock_context.add_expected_status_query_result( + self.create_mock_status_query_result(ScheduleStatus.INIT)) + mock_context.add_expected_status_query_result( + self.create_mock_status_query_result(ScheduleStatus.RUNNING)) + api = mock_context.get_api('west') + api.create_job.return_value = self.get_createjob_response() + + # This is the real test: invoke create as if it had been called by the command line. + with temporary_file() as fp: + fp.write(self.get_valid_config()) + fp.flush() + cmd = AuroraCommandLine() + cmd.execute(['job', 'create', '--wait_until=RUNNING', 'west/mchucarroll/test/hello', + fp.name]) + + # Now check that the right API calls got made. + # Check that create_job was called exactly once, with an AuroraConfig parameter. + self.assert_create_job_called(api) + self.assert_scheduler_called(api, mock_query, 2) + + def test_create_job_delayed(self): + """Run a test of the "create" command against a mocked-out API: + this time, make the monitor check status several times before successful completion. + """ + mock_context = FakeAuroraCommandContext() + with contextlib.nested( + patch('time.sleep'), + patch('twitter.aurora.client.cli.jobs.Job.create_context', return_value=mock_context)): + mock_query = self.create_mock_query() + for result in [ScheduleStatus.INIT, ScheduleStatus.PENDING, ScheduleStatus.PENDING, + ScheduleStatus.RUNNING, ScheduleStatus.FINISHED]: + mock_context.add_expected_status_query_result(self.create_mock_status_query_result(result)) + api = mock_context.get_api('west') + api.create_job.return_value = self.get_createjob_response() + with temporary_file() as fp: + fp.write(self.get_valid_config()) + fp.flush() + cmd = AuroraCommandLine() + cmd.execute(['job', 'create', '--wait_until=RUNNING', 'west/mchucarroll/test/hello', + fp.name]) + # Now check that the right API calls got made. + # Check that create_job was called exactly once, with an AuroraConfig parameter. + self.assert_create_job_called(api) + self.assert_scheduler_called(api, mock_query, 4) + + def test_create_job_failed(self): + """Run a test of the "create" command against a mocked-out API: + this time, make the monitor check status several times before successful completion. + """ + mock_context = FakeAuroraCommandContext() + with patch('twitter.aurora.client.cli.jobs.Job.create_context', return_value=mock_context): + mock_context.add_expected_status_query_result( + self.create_mock_status_query_result(ScheduleStatus.INIT)) + api = mock_context.get_api('west') + api.create_job.return_value = self.get_failed_createjob_response() + # This is the real test: invoke create as if it had been called by the command line. + with temporary_file() as fp: + fp.write(self.get_valid_config()) + fp.flush() + cmd = AuroraCommandLine() + result = cmd.execute(['job', 'create', '--wait_until=RUNNING', + 'west/mchucarroll/test/hello', fp.name]) + assert result == EXIT_NETWORK_ERROR + + # Now check that the right API calls got made. + # Check that create_job was called exactly once, with an AuroraConfig parameter. + self.assert_create_job_called(api) + + # getTasksStatus was called once, before the create_job + assert api.scheduler.getTasksStatus.call_count == 1 + + def test_create_job_failed_invalid_config(self): + """Run a test of the "create" command against a mocked-out API, with a configuration + containing a syntax error""" + mock_context = FakeAuroraCommandContext() + with patch('twitter.aurora.client.cli.jobs.Job.create_context', return_value=mock_context): + with temporary_file() as fp: + fp.write(self.get_invalid_config('invalid_clause=oops')) + fp.flush() + cmd = AuroraCommandLine() + result = cmd.execute(['job', 'create', '--wait_until=RUNNING', + 'west/mchucarroll/test/hello', fp.name]) + assert result == EXIT_INVALID_CONFIGURATION + + # Now check that the right API calls got made. + # Check that create_job was not called. + api = mock_context.get_api('west') + assert api.create_job.call_count == 0 + assert api.scheduler.getTasksStatus.call_count == 0 http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/39a94c1a/src/test/python/twitter/aurora/client/cli/test_kill.py ---------------------------------------------------------------------- diff --git a/src/test/python/twitter/aurora/client/cli/test_kill.py b/src/test/python/twitter/aurora/client/cli/test_kill.py new file mode 100644 index 0000000..8f8095c --- /dev/null +++ b/src/test/python/twitter/aurora/client/cli/test_kill.py @@ -0,0 +1,114 @@ +import contextlib +import unittest + +from twitter.aurora.client.cli import AuroraCommandLine +from twitter.aurora.client.hooks.hooked_api import HookedAuroraClientAPI +from twitter.aurora.common.aurora_job_key import AuroraJobKey +from twitter.common.contextutil import temporary_file +from twitter.aurora.client.cli.jobs import parse_instances +from twitter.aurora.client.cli.util import AuroraClientCommandTest, FakeAuroraCommandContext + +from gen.twitter.aurora.ttypes import ( + Identity, + TaskQuery, +) + +from mock import Mock, patch + + +class TestInstancesParser(unittest.TestCase): + def test_parse_instances(self): + instances = '0,1-3,5' + x = parse_instances(instances) + assert x == [0, 1, 2, 3, 5] + + def test_parse_none(self): + assert parse_instances(None) is None + assert parse_instances("") is None + + +class TestClientKillCommand(AuroraClientCommandTest): + @classmethod + def setup_mock_api(cls): + """Builds up a mock API object, with a mock SchedulerProxy. + Returns the API and the proxy""" + + mock_scheduler = Mock() + mock_scheduler.url = "http://something_or_other" + mock_scheduler_client = Mock() + mock_scheduler_client.scheduler.return_value = mock_scheduler + mock_scheduler_client.url = "http://something_or_other" + mock_api = Mock(spec=HookedAuroraClientAPI) + mock_api.scheduler = mock_scheduler_client + return (mock_api, mock_scheduler_client) + + @classmethod + def get_kill_job_response(cls): + return cls.create_simple_success_response() + + @classmethod + def assert_kill_job_called(cls, mock_api): + assert mock_api.kill_job.call_count == 1 + + def test_kill_job(self): + """Test kill client-side API logic.""" + mock_context = FakeAuroraCommandContext() + mock_scheduler = Mock() + with contextlib.nested( + patch('twitter.aurora.client.cli.jobs.Job.create_context', return_value=mock_context), + patch('twitter.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS)): + + api = mock_context.get_api('west') + api.kill_job.return_value = self.get_kill_job_response() + mock_scheduler.scheduler.killTasks.return_value = self.get_kill_job_response() + with temporary_file() as fp: + fp.write(self.get_valid_config()) + fp.flush() + cmd = AuroraCommandLine() + cmd.execute(['job', 'kill', '--config=%s' % fp.name, 'west/mchucarroll/test/hello']) + + # Now check that the right API calls got made. + assert api.kill_job.call_count == 1 + api.kill_job.assert_called_with(AuroraJobKey.from_path('west/mchucarroll/test/hello'), None) + + def test_kill_job_with_instances(self): + """Test kill client-side API logic.""" + mock_context = FakeAuroraCommandContext() + with contextlib.nested( + patch('twitter.aurora.client.cli.jobs.Job.create_context', return_value=mock_context), + patch('twitter.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS)): + api = mock_context.get_api('west') + api.kill_job.return_value = self.get_kill_job_response() + with temporary_file() as fp: + fp.write(self.get_valid_config()) + fp.flush() + cmd = AuroraCommandLine() + cmd.execute(['job', 'kill', '--config=%s' % fp.name, '--instances=0,2,4-6', + 'west/mchucarroll/test/hello']) + + # Now check that the right API calls got made. + assert api.kill_job.call_count == 1 + api.kill_job.assert_called_with(AuroraJobKey.from_path('west/mchucarroll/test/hello'), + [0, 2, 4, 5, 6]) + + def test_kill_job_with_instances_deep_api(self): + """Test kill client-side API logic.""" + mock_context = FakeAuroraCommandContext() + (mock_api, mock_scheduler) = self.setup_mock_api() + with contextlib.nested( + patch('twitter.aurora.client.api.SchedulerProxy', return_value=mock_scheduler), + patch('twitter.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS)): + mock_scheduler.killTasks.return_value = self.get_kill_job_response() + with temporary_file() as fp: + fp.write(self.get_valid_config()) + fp.flush() + cmd = AuroraCommandLine() + cmd.execute(['job', 'kill', '--config=%s' % fp.name, '--instances=0,2,4-6', + 'west/mchucarroll/test/hello']) + # Now check that the right API calls got made. + assert mock_scheduler.killTasks.call_count == 1 + mock_scheduler.killTasks.assert_called_with( + TaskQuery(jobName='hello', environment='test', instanceIds=frozenset([0, 2, 4, 5, 6]), + owner=Identity(role='mchucarroll')), None) + + http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/39a94c1a/src/test/python/twitter/aurora/client/cli/util.py ---------------------------------------------------------------------- diff --git a/src/test/python/twitter/aurora/client/cli/util.py b/src/test/python/twitter/aurora/client/cli/util.py new file mode 100644 index 0000000..46883ec --- /dev/null +++ b/src/test/python/twitter/aurora/client/cli/util.py @@ -0,0 +1,152 @@ +import unittest + +from twitter.aurora.client.cli.context import AuroraCommandContext +from twitter.aurora.client.hooks.hooked_api import HookedAuroraClientAPI +from twitter.aurora.common.cluster import Cluster +from twitter.aurora.common.clusters import Clusters + +from gen.twitter.aurora.ttypes import ( + Response, + ResponseCode, + Result, +) + +from mock import Mock + + +class FakeAuroraCommandContext(AuroraCommandContext): + def __init__(self): + super(FakeAuroraCommandContext, self).__init__() + self.options = None + self.status = [] + self.fake_api = self.setup_fake_api() + self.task_status = [] + self.showed_urls = [] + + def get_api(self, cluster): + return self.fake_api + + def setup_fake_api(self): + # In here, we'd like to get it mocked so that the HookedAuroraClientAPI + # object, and its underlying AuroraClientAPI objects are not + # mocked, but the scheduler object is. + new_fake = Mock(spec=HookedAuroraClientAPI) + new_fake.scheduler = Mock() + new_fake.scheduler.url = 'http://something_or_other' + new_fake.scheduler.getTasksStatus.side_effect = [] + self.fake_api = new_fake + return self.fake_api + + def open_page(self, url): + self.showed_urls.append(url) + + def handle_open(self, api): + pass + + def add_expected_status_query_result(self, expected_result): + self.task_status.append(expected_result) + # each call adds an expected query result, in order. + self.fake_api.scheduler.getTasksStatus.side_effect = self.task_status + + +class AuroraClientCommandTest(unittest.TestCase): + @classmethod + def create_blank_response(cls, code, msg): + response = Mock(spec=Response) + response.responseCode = code + response.message = msg + response.result = Mock(spec=Result) + return response + + @classmethod + def create_simple_success_response(cls): + return cls.create_blank_response(ResponseCode.OK, 'OK') + + @classmethod + def create_error_response(cls): + return cls.create_blank_response(ResponseCode.ERROR, 'Damn') + + @classmethod + def create_mock_api(cls): + """Builds up a mock API object, with a mock SchedulerProxy""" + mock_api = Mock(spec=HookedAuroraClientAPI) + mock_scheduler = Mock() + mock_scheduler.url = "http://something_or_other" + mock_scheduler_client = Mock() + mock_scheduler_client.scheduler.return_value = mock_scheduler + mock_scheduler_client.url = "http://something_or_other" + mock_api = Mock(spec=HookedAuroraClientAPI) + mock_api.scheduler = mock_scheduler_client + return (mock_api, mock_scheduler_client) + + @classmethod + def create_mock_api_factory(cls): + """Create a collection of mocks for a test that wants to mock out the client API + by patching the api factory.""" + mock_api, mock_scheduler_client = cls.create_mock_api() + mock_api_factory = Mock() + mock_api_factory.return_value = mock_api + return mock_api_factory, mock_scheduler_client + + FAKE_TIME = 42131 + + @classmethod + def fake_time(cls, ignored): + """Utility function used for faking time to speed up tests.""" + cls.FAKE_TIME += 2 + return cls.FAKE_TIME + + CONFIG_BASE = """ +HELLO_WORLD = Job( + name = '%(job)s', + role = '%(role)s', + cluster = '%(cluster)s', + environment = '%(env)s', + instances = 20, + %(inner)s + update_config = UpdateConfig( + batch_size = 5, + restart_threshold = 30, + watch_secs = 10, + max_per_shard_failures = 2, + ), + task = Task( + name = 'test', + processes = [Process(name = 'hello_world', cmdline = 'echo {{thermos.ports[http]}}')], + resources = Resources(cpu = 0.1, ram = 64 * MB, disk = 64 * MB), + ) +) +jobs = [HELLO_WORLD] +""" + + TEST_ROLE = 'mchucarroll' + + TEST_ENV = 'test' + + TEST_JOB = 'hello' + + TEST_CLUSTER = 'west' + + TEST_JOBSPEC = 'west/mchucarroll/test/hello' + + TEST_CLUSTERS = Clusters([Cluster( + name='west', + packer_copy_command='copying {{package}}', + zk='zookeeper.example.com', + scheduler_zk_path='/foo/bar', + auth_mechanism='UNAUTHENTICATED')]) + + @classmethod + def get_test_config(cls, cluster, role, env, job, filler=''): + """Create a config from the template""" + return cls.CONFIG_BASE % {'job': job, 'role': role, 'env': env, 'cluster': cluster, + 'inner': filler} + + @classmethod + def get_valid_config(cls): + return cls.get_test_config(cls.TEST_CLUSTER, cls.TEST_ROLE, cls.TEST_ENV, cls.TEST_JOB) + + @classmethod + def get_invalid_config(cls, bad_clause): + return cls.get_test_config(cls.TEST_CLUSTER, cls.TEST_ROLE, cls.TEST_ENV, cls.TEST_JOB, + bad_clause) http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/39a94c1a/src/test/python/twitter/aurora/client/commands/test_kill.py ---------------------------------------------------------------------- diff --git a/src/test/python/twitter/aurora/client/commands/test_kill.py b/src/test/python/twitter/aurora/client/commands/test_kill.py index 3649969..a0cc716 100644 --- a/src/test/python/twitter/aurora/client/commands/test_kill.py +++ b/src/test/python/twitter/aurora/client/commands/test_kill.py @@ -19,7 +19,7 @@ from gen.twitter.aurora.ttypes import ( from mock import Mock, patch -class TestClientKllCommand(AuroraClientCommandTest): +class TestClientKillCommand(AuroraClientCommandTest): @classmethod def setup_mock_options(cls):