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/05/29 16:39:04 UTC

git commit: Adding cron client commands.

Repository: incubator-aurora
Updated Branches:
  refs/heads/master eb1155d49 -> d92f86c8c


Adding cron client commands.

Add cron schedule and deschedule calls to the scheduler API.

Bugs closed: aurora-407

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


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

Branch: refs/heads/master
Commit: d92f86c8cc49fff4ac5ca8a679e74bba34edc39d
Parents: eb1155d
Author: Mark Chu-Carroll <mc...@twopensource.com>
Authored: Thu May 29 10:34:02 2014 -0400
Committer: Mark Chu-Carroll <mc...@apache.org>
Committed: Thu May 29 10:34:02 2014 -0400

----------------------------------------------------------------------
 .../python/apache/aurora/client/api/__init__.py |  10 ++
 src/main/python/apache/aurora/client/cli/BUILD  |   1 +
 .../python/apache/aurora/client/cli/bridge.py   |   7 +-
 .../python/apache/aurora/client/cli/client.py   |   2 +
 .../python/apache/aurora/client/cli/cron.py     | 163 +++++++++++++++++++
 .../python/apache/aurora/client/cli/options.py  |   2 +
 src/test/python/apache/aurora/client/cli/BUILD  |  17 ++
 .../apache/aurora/client/cli/test_cron.py       | 154 ++++++++++++++++++
 8 files changed, 352 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d92f86c8/src/main/python/apache/aurora/client/api/__init__.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/api/__init__.py b/src/main/python/apache/aurora/client/api/__init__.py
index a23660b..1af0f8f 100644
--- a/src/main/python/apache/aurora/client/api/__init__.py
+++ b/src/main/python/apache/aurora/client/api/__init__.py
@@ -62,6 +62,16 @@ class AuroraClientAPI(object):
     log.debug('Lock %s' % lock)
     return self._scheduler_proxy.createJob(config.job(), lock)
 
+  def schedule_cron(self, config, lock=None):
+    log.info("Registering job %s with cron" % config.name())
+    log.debug('Full configuration: %s' % config.job())
+    log.debug('Lock %s' % lock)
+    return self._scheduler_proxy.scheduleCronJob(config.job(), lock)
+
+  def deschedule_cron(self, jobkey):
+    log.info("Removing cron schedule for job %s" % jobkey)
+    return self._scheduler_proxy.descheduleCronJob(jobkey.to_thrift())
+
   def populate_job_config(self, config):
     return self._scheduler_proxy.populateJobConfig(config.job())
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d92f86c8/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 015345e..b3b8661 100644
--- a/src/main/python/apache/aurora/client/cli/BUILD
+++ b/src/main/python/apache/aurora/client/cli/BUILD
@@ -43,6 +43,7 @@ python_library(
     'config.py',
     'context.py',
     'command_hooks.py',
+    'cron.py',
     'jobs.py',
     'options.py',
     'quota.py',

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d92f86c8/src/main/python/apache/aurora/client/cli/bridge.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/bridge.py b/src/main/python/apache/aurora/client/cli/bridge.py
index 6604ee5..d5eec8a 100644
--- a/src/main/python/apache/aurora/client/cli/bridge.py
+++ b/src/main/python/apache/aurora/client/cli/bridge.py
@@ -65,15 +65,14 @@ class Bridge(object):
       for cp in self.command_processors:
         print("========== help for %s ==========" % cp.name)
         cp.execute(args)
-      return
+      return 0
     elif len(args) >= 3:
       discriminator = args[2]
       for cp in self.command_processors:
         if discriminator in cp.get_commands():
-          cp.execute(args)
-          return
+          return cp.execute(args)
       if self.default is not None:
-        self.default.execute(args)
+        return self.default.execute(args)
 
 
   def execute(self, args):

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d92f86c8/src/main/python/apache/aurora/client/cli/client.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/client.py b/src/main/python/apache/aurora/client/cli/client.py
index 12e7bcf..ab0a7ef 100644
--- a/src/main/python/apache/aurora/client/cli/client.py
+++ b/src/main/python/apache/aurora/client/cli/client.py
@@ -26,6 +26,8 @@ class AuroraCommandLine(CommandLine):
 
   def register_nouns(self):
     super(AuroraCommandLine, self).register_nouns()
+    from apache.aurora.client.cli.cron import CronNoun
+    self.register_noun(CronNoun())
     from apache.aurora.client.cli.jobs import Job
     self.register_noun(Job())
     from apache.aurora.client.cli.config import ConfigNoun

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d92f86c8/src/main/python/apache/aurora/client/cli/cron.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/cron.py b/src/main/python/apache/aurora/client/cli/cron.py
new file mode 100644
index 0000000..c30a0a6
--- /dev/null
+++ b/src/main/python/apache/aurora/client/cli/cron.py
@@ -0,0 +1,163 @@
+#
+# Copyright 2014 Apache Software Foundation
+#
+# 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.
+#
+
+from __future__ import print_function
+
+from apache.aurora.client.cli import (
+    EXIT_COMMAND_FAILURE,
+    EXIT_INVALID_PARAMETER,
+    EXIT_OK,
+    Noun,
+    Verb,
+)
+from apache.aurora.client.cli.context import AuroraCommandContext
+from apache.aurora.client.cli.options import (
+    BIND_OPTION,
+    BROWSER_OPTION,
+    CONFIG_ARGUMENT,
+    CONFIG_OPTION,
+    JOBSPEC_ARGUMENT,
+    JSON_READ_OPTION,
+)
+
+from gen.apache.aurora.api.ttypes import ResponseCode
+
+
+class Schedule(Verb):
+  @property
+  def name(self):
+    return 'schedule'
+
+  @property
+  def help(self):
+    return "Create a cron schedule for a job."
+
+  def get_options(self):
+    return [BIND_OPTION, JSON_READ_OPTION, JOBSPEC_ARGUMENT, CONFIG_ARGUMENT]
+
+  def execute(self, context):
+    api = context.get_api(context.options.jobspec.cluster)
+    config = context.get_job_config(context.options.jobspec, context.options.config_file)
+    resp = api.schedule_cron(config)
+    if resp.responseCode != ResponseCode.OK:
+      context.print_err("Error scheduling job %s: %s" % (context.options.jobspec,
+          resp.messageDEPRECATED))
+      return EXIT_COMMAND_FAILURE
+    else:
+      return EXIT_OK
+
+
+class Deschedule(Verb):
+  @property
+  def name(self):
+    return 'deschedule'
+
+  @property
+  def help(self):
+    return "Remove the cron schedule for a job."
+
+  def get_options(self):
+    return [BIND_OPTION, JOBSPEC_ARGUMENT]
+
+  def execute(self, context):
+    api = context.get_api(context.options.jobspec.cluster)
+    resp = api.deschedule_cron(context.options.jobspec)
+    if resp.responseCode != ResponseCode.OK:
+      context.print_err("Error descheduling job %s: %s" % (context.options.jobspec,
+          resp.messageDEPRECATED))
+      return EXIT_COMMAND_FAILURE
+    else:
+      return EXIT_OK
+
+
+class Start(Verb):
+  @property
+  def name(self):
+    return 'start'
+
+  @property
+  def help(self):
+    return """Start a cron job immediately, outside of its normal cron schedule."""
+
+  def get_options(self):
+    return [BIND_OPTION, BROWSER_OPTION, CONFIG_OPTION, JSON_READ_OPTION, JOBSPEC_ARGUMENT]
+
+  def execute(self, context):
+    api = context.get_api(context.options.jobspec.cluster)
+    config = (context.get_job_config(context.options.jobspec, context.options.config)
+        if context.options.config else None)
+    resp = api.start_cronjob(context.options.jobspec, config=config)
+    if resp.responseCode != ResponseCode.OK:
+      context.print_err("Error starting cron job %s: %s" % (context.options.jobspec,
+          resp.messageDEPRECATED))
+      return EXIT_COMMAND_FAILURE
+    if context.options.open_browser:
+      context.open_job_page(api, context.options.job_spec)
+    return EXIT_OK
+
+
+class Show(Verb):
+  @property
+  def name(self):
+    return 'show'
+
+  @property
+  def help(self):
+    return """Get the scheduling status of a cron job"""
+
+  def get_options(self):
+    return [JOBSPEC_ARGUMENT]
+
+  def execute(self, context):
+    #TODO(mchucarroll): do we want to support wildcards here?
+    jobkey = context.options.jobspec
+    api = context.get_api(jobkey.cluster)
+    resp = api.get_jobs(jobkey.role)
+    if resp.responseCode != ResponseCode.OK:
+      context.print_err("Error getting cron status for %s: %s" % (jobkey, resp.messageDEPRECATED))
+    for job in resp.result.getJobsResult.configs:
+      if job.key.environment == jobkey.env and job.key.name == jobkey.name:
+        if job.cronSchedule is None or job.cronSchedule == "":
+          context.print_err("No cron entry found for job %s" % jobkey)
+          return EXIT_INVALID_PARAMETER
+        else:
+          context.print_out('%s\t %s' % (jobkey, job.cronSchedule))
+          return EXIT_OK
+      else:
+        context.print_err("No cron entry found for job %s" % jobkey)
+        return EXIT_INVALID_PARAMETER
+    return EXIT_OK
+
+
+class CronNoun(Noun):
+  @property
+  def name(self):
+    return 'cron'
+
+  @property
+  def help(self):
+    return "Work with entries in the aurora cron scheduler"
+
+  @classmethod
+  def create_context(cls):
+    return AuroraCommandContext()
+
+  def __init__(self):
+    super(CronNoun, self).__init__()
+    self.register_verb(Schedule())
+    self.register_verb(Deschedule())
+    self.register_verb(Start())
+    self.register_verb(Show())

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d92f86c8/src/main/python/apache/aurora/client/cli/options.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/options.py b/src/main/python/apache/aurora/client/cli/options.py
index d79b4dc..0d49bac 100644
--- a/src/main/python/apache/aurora/client/cli/options.py
+++ b/src/main/python/apache/aurora/client/cli/options.py
@@ -203,6 +203,8 @@ BROWSER_OPTION = CommandOption('--open-browser', default=False, dest='open_brows
 CONFIG_ARGUMENT = CommandOption('config_file', type=str,
     help='pathname of the aurora configuration file contain the job specification')
 
+CONFIG_OPTION = CommandOption('--config', type=str, default=None,
+    help='pathname of the aurora configuration file contain the job specification')
 
 EXECUTOR_SANDBOX_OPTION = CommandOption('--executor-sandbox', action='store_true',
      default=False, help='Run the command in the executor sandbox instead of the task sandbox')

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d92f86c8/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 5667788..c91ae42 100644
--- a/src/test/python/apache/aurora/client/cli/BUILD
+++ b/src/test/python/apache/aurora/client/cli/BUILD
@@ -17,6 +17,7 @@ python_test_suite(
   dependencies = [
     pants(':bridge'),
     pants(':command_hooks'),
+    pants(':cron'),
     pants(':help'),
     pants(':job'),
     pants(':config'),
@@ -117,6 +118,22 @@ python_tests(
 )
 
 python_tests(
+  name = 'cron',
+  sources = [
+    'test_cron.py',
+  ],
+  dependencies = [
+    pants(':util'),
+    pants('3rdparty/python:mock'),
+    pants('3rdparty/python:twitter.common.contextutil'),
+    pants('src/main/python/apache/aurora/client/cli'),
+    pants('src/main/python/apache/aurora/client/cli:client_lib'),
+    pants('src/test/python/apache/aurora/client/commands:util')
+  ]
+)
+
+
+python_tests(
   name = 'quota',
   sources = [ 'test_quota.py' ],
   dependencies = [

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d92f86c8/src/test/python/apache/aurora/client/cli/test_cron.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/test_cron.py b/src/test/python/apache/aurora/client/cli/test_cron.py
new file mode 100644
index 0000000..049405a
--- /dev/null
+++ b/src/test/python/apache/aurora/client/cli/test_cron.py
@@ -0,0 +1,154 @@
+#
+# Copyright 2013 Apache Software Foundation
+#
+# 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 contextlib
+from mock import Mock, patch
+
+from apache.aurora.client.cli import EXIT_COMMAND_FAILURE, EXIT_INVALID_CONFIGURATION, EXIT_OK
+from apache.aurora.client.cli.client import AuroraCommandLine
+from apache.aurora.client.cli.util import AuroraClientCommandTest, FakeAuroraCommandContext
+from apache.aurora.config import AuroraConfig
+
+from gen.apache.aurora.api.ttypes import JobKey
+
+
+from twitter.common.contextutil import temporary_file
+
+
+class TestCronNoun(AuroraClientCommandTest):
+
+  def test_successful_schedule(self):
+    mock_context = FakeAuroraCommandContext()
+    with contextlib.nested(
+        patch('apache.aurora.client.cli.cron.CronNoun.create_context', return_value=mock_context)):
+
+      api = mock_context.get_api('west')
+      api.schedule_cron.return_value = self.create_simple_success_response()
+      with temporary_file() as fp:
+        fp.write(self.get_valid_config())
+        fp.flush()
+        cmd = AuroraCommandLine()
+        cmd.execute(['cron', 'schedule', 'west/bozo/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.
+      assert api.schedule_cron.call_count == 1
+      assert isinstance(api.schedule_cron.call_args[0][0], AuroraConfig)
+
+  def test_schedule_failed(self):
+    mock_context = FakeAuroraCommandContext()
+    with patch('apache.aurora.client.cli.cron.CronNoun.create_context', return_value=mock_context):
+      api = mock_context.get_api('west')
+      api.schedule_cron.return_value = self.create_error_response()
+      with temporary_file() as fp:
+        fp.write(self.get_valid_config())
+        fp.flush()
+        cmd = AuroraCommandLine()
+        result = cmd.execute(['cron', 'schedule', 'west/bozo/test/hello', fp.name])
+        assert result == EXIT_COMMAND_FAILURE
+
+      # Now check that the right API calls got made.
+      # Check that create_job was called exactly once, with an AuroraConfig parameter.
+      assert api.schedule_cron.call_count == 1
+
+  def test_schedule_cron_failed_invalid_config(self):
+    mock_context = FakeAuroraCommandContext()
+    with patch('apache.aurora.client.cli.cron.CronNoun.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(['cron', 'schedule', 'west/bozo/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.schedule_cron.call_count == 0
+
+  def test_schedule_cron_deep_api(self):
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
+    with contextlib.nested(
+        patch('time.sleep'),
+        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
+        patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS)):
+      mock_scheduler_proxy.scheduleCronJob.return_value = self.create_simple_success_response()
+      with temporary_file() as fp:
+        fp.write(self.get_valid_config())
+        fp.flush()
+        cmd = AuroraCommandLine()
+        result = cmd.execute(['cron', 'schedule', 'west/bozo/test/hello', fp.name])
+        assert result == EXIT_OK
+        assert mock_scheduler_proxy.scheduleCronJob.call_count == 1
+        job = mock_scheduler_proxy.scheduleCronJob.call_args[0][0]
+        assert job.key == JobKey("bozo", "test", "hello")
+
+  def test_deschedule_cron_deep_api(self):
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
+    with contextlib.nested(
+        patch('time.sleep'),
+        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
+        patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS)):
+      mock_scheduler_proxy.descheduleCronJob.return_value = self.create_simple_success_response()
+      cmd = AuroraCommandLine()
+      result = cmd.execute(['cron', 'deschedule', 'west/bozo/test/hello'])
+      assert result == EXIT_OK
+      assert mock_scheduler_proxy.descheduleCronJob.call_count == 1
+      mock_scheduler_proxy.descheduleCronJob.assert_called_with(JobKey(environment='test',
+          role='bozo', name='hello'))
+
+  def test_start_cron(self):
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
+    with contextlib.nested(
+        patch('time.sleep'),
+        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
+        patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS)):
+      mock_scheduler_proxy.startCronJob.return_value = self.create_simple_success_response()
+      cmd = AuroraCommandLine()
+      result = cmd.execute(['cron', 'start', 'west/bozo/test/hello'])
+      assert result == EXIT_OK
+      mock_scheduler_proxy.startCronJob.assert_called_once_with(JobKey("bozo", "test", "hello"))
+
+  @classmethod
+  def _create_getjobs_response(cls):
+    response = cls.create_simple_success_response()
+    response.result = Mock()
+    response.result.getJobsResult = Mock()
+    mockjob = Mock()
+    mockjob.cronSchedule = "* * * * *"
+    mockjob.key = Mock()
+    mockjob.key.environment = "test"
+    mockjob.key.name = "hello"
+    mockjob.key.role = "bozo"
+    response.result.getJobsResult.configs = [mockjob]
+    return response
+
+  def test_cron_status(self):
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
+    with contextlib.nested(
+        patch('time.sleep'),
+        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.context.AuroraCommandContext.print_out')) as (
+            _, _, _, mock_print):
+      mock_scheduler_proxy.getJobs.return_value = self._create_getjobs_response()
+      cmd = AuroraCommandLine()
+      result = cmd.execute(['cron', 'show', 'west/bozo/test/hello'])
+
+      assert result == EXIT_OK
+      mock_scheduler_proxy.getJobs.assert_called_once_with("bozo")
+      mock_print.assert_called_with("west/bozo/test/hello\t * * * * *")