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/21 15:14:07 UTC

git commit: Add cron schedule and deschedule calls to the scheduler API.

Repository: incubator-aurora
Updated Branches:
  refs/heads/master 39f18458d -> 8f3b20404


Add cron schedule and deschedule calls to the scheduler API.

Bugs closed: aurora-417

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


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

Branch: refs/heads/master
Commit: 8f3b20404269a812ee12519e0a621d04838471c6
Parents: 39f1845
Author: Mark Chu-Carroll <mc...@twopensource.com>
Authored: Wed May 21 09:09:57 2014 -0400
Committer: Mark Chu-Carroll <mc...@apache.org>
Committed: Wed May 21 09:09:57 2014 -0400

----------------------------------------------------------------------
 .../scheduler/state/SchedulerCoreImpl.java      |  7 +-
 .../thrift/SchedulerThriftInterface.java        | 95 ++++++++++++++++++++
 .../python/apache/aurora/client/cli/__init__.py | 10 +--
 .../thrift/org/apache/aurora/gen/api.thrift     |  9 ++
 .../thrift/SchedulerThriftInterfaceTest.java    | 38 ++++++++
 .../scheduler/thrift/aop/ForwardingThrift.java  | 15 +++-
 .../aurora/client/api/test_scheduler_client.py  | 10 +++
 7 files changed, 177 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8f3b2040/src/main/java/org/apache/aurora/scheduler/state/SchedulerCoreImpl.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/state/SchedulerCoreImpl.java b/src/main/java/org/apache/aurora/scheduler/state/SchedulerCoreImpl.java
index d377974..fa52c33 100644
--- a/src/main/java/org/apache/aurora/scheduler/state/SchedulerCoreImpl.java
+++ b/src/main/java/org/apache/aurora/scheduler/state/SchedulerCoreImpl.java
@@ -142,9 +142,11 @@ class SchedulerCoreImpl implements SchedulerCore {
         }
 
         validateTaskLimits(job.getTaskConfig(), job.getInstanceCount());
-
+        // TODO(mchucarroll): deprecate cron as a part of create/kill job.(AURORA-454)
         if (isCron(sanitizedConfiguration)) {
           try {
+            LOG.warning("Deprecated behavior: scheduling job " + job.getKey()
+                + " with cron via createJob (AURORA_454)");
             cronJobManager.createJob(SanitizedCronJob.from(sanitizedConfiguration));
           } catch (CronException e) {
             throw new ScheduleException(e);
@@ -239,7 +241,10 @@ class SchedulerCoreImpl implements SchedulerCore {
       // If this looks like a query for all tasks in a job, instruct the cron scheduler to delete
       // it.
       // TODO(maxim): Should be trivial to support killing multiple jobs instead.
+      // TODO(mchucarroll): deprecate cron as a part of create/kill job.  (AURORA-454)
       IJobKey jobKey = Iterables.getOnlyElement(JobKeys.from(query).get());
+      LOG.warning("Deprecated behavior: descheduling job " + jobKey
+          + " with cron via killTasks. (See AURORA-454)");
       cronJobManager.deleteJob(jobKey);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8f3b2040/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java b/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java
index 4386a86..71b3880 100644
--- a/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java
+++ b/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java
@@ -251,7 +251,102 @@ class SchedulerThriftInterface implements AuroraAdmin.Iface {
     } catch (TaskDescriptionException | ScheduleException e) {
       response.setResponseCode(INVALID_REQUEST).setMessage(e.getMessage());
     }
+    return response;
+  }
+
+  private static boolean isCron(SanitizedConfiguration config) {
+    if (!config.getJobConfig().isSetCronSchedule()) {
+      return false;
+    } else if (StringUtils.isEmpty(config.getJobConfig().getCronSchedule())) {
+      // TODO(ksweeney): Remove this in 0.7.0 (AURORA-424).
+      LOG.warning("Got service config with empty string cron schedule. aurora-0.7.x "
+          + "will interpret this as cron job and cause an error.");
+      return false;
+    } else {
+      return true;
+    }
+  }
+
+  @Override
+  public Response scheduleCronJob(
+      JobConfiguration mutableJob,
+      @Nullable Lock mutableLock,
+      SessionKey session) {
+
+    IJobConfiguration job = IJobConfiguration.build(mutableJob);
+    IJobKey jobKey = JobKeys.assertValid(job.getKey());
+    checkNotNull(session);
+
+    Response response = new Response();
+
+    try {
+      sessionValidator.checkAuthenticated(session, ImmutableSet.of(job.getOwner().getRole()));
+    } catch (AuthFailedException e) {
+      return response.setResponseCode(AUTH_FAILED).setMessage(e.getMessage());
+    }
+
+    try {
+      SanitizedConfiguration sanitized = SanitizedConfiguration.fromUnsanitized(job);
+
+      lockManager.validateIfLocked(
+          ILockKey.build(LockKey.job(jobKey.newBuilder())),
+          Optional.fromNullable(mutableLock).transform(ILock.FROM_BUILDER));
 
+      if (!isCron(sanitized)) {
+        LOG.info("Invalid attempt to schedule non-cron job "
+            + sanitized.getJobConfig().getKey()
+            + " with cron.");
+        response.setResponseCode(INVALID_REQUEST)
+            .setMessage("Job " + sanitized.getJobConfig().getKey()
+            + " has no cron schedule");
+        return response;
+      }
+      try {
+        // TODO(mchucarroll): Merge CronJobManager.createJob/updateJob
+        if (cronJobManager.hasJob(sanitized.getJobConfig().getKey())) {
+          // The job already has a schedule: so update it.
+          cronJobManager.updateJob(SanitizedCronJob.from(sanitized));
+        } else {
+          cronJobManager.createJob(SanitizedCronJob.from(sanitized));
+        }
+      } catch (CronException e) {
+        response.setResponseCode(INVALID_REQUEST).setMessage(e.getMessage());
+        return response;
+      }
+      response.setResponseCode(OK)
+          .setMessage(String.format("Job %s scheduled with cron schedule '%s",
+               sanitized.getJobConfig().getKey(), sanitized.getJobConfig().getCronSchedule()));
+    } catch (LockException e) {
+      response.setResponseCode(LOCK_ERROR).setMessage(e.getMessage());
+    } catch (TaskDescriptionException e) {
+      response.setResponseCode(INVALID_REQUEST).setMessage(e.getMessage());
+    }
+    return response;
+  }
+
+  @Override
+  public Response descheduleCronJob(
+      JobKey mutableJobKey,
+      @Nullable Lock mutableLock,
+      SessionKey session) {
+
+    Response response = new Response();
+    try {
+      IJobKey jobKey = JobKeys.assertValid(IJobKey.build(mutableJobKey));
+      lockManager.validateIfLocked(
+          ILockKey.build(LockKey.job(jobKey.newBuilder())),
+          Optional.fromNullable(mutableLock).transform(ILock.FROM_BUILDER));
+
+      if (!cronJobManager.deleteJob(jobKey)) {
+        response.setResponseCode(INVALID_REQUEST)
+            .setMessage("Job " + jobKey + " is not scheduled with cron");
+        return response;
+      }
+      response.setResponseCode(OK)
+          .setMessage(String.format("Job %s removed from cron schedule", jobKey));
+    } catch (LockException e) {
+      response.setResponseCode(INVALID_REQUEST).setMessage(e.getMessage());
+    }
     return response;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8f3b2040/src/main/python/apache/aurora/client/cli/__init__.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/__init__.py b/src/main/python/apache/aurora/client/cli/__init__.py
index 2beb9a0..b4d43c0 100644
--- a/src/main/python/apache/aurora/client/cli/__init__.py
+++ b/src/main/python/apache/aurora/client/cli/__init__.py
@@ -16,11 +16,11 @@
 
 '''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.
+This module provides a framework for a noun/verb command-line application.
+In this framework, an 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":

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8f3b2040/src/main/thrift/org/apache/aurora/gen/api.thrift
----------------------------------------------------------------------
diff --git a/src/main/thrift/org/apache/aurora/gen/api.thrift b/src/main/thrift/org/apache/aurora/gen/api.thrift
index 66292dc..aad74fd 100644
--- a/src/main/thrift/org/apache/aurora/gen/api.thrift
+++ b/src/main/thrift/org/apache/aurora/gen/api.thrift
@@ -502,6 +502,15 @@ service AuroraSchedulerManager extends ReadOnlyScheduler {
   // name already exists in the cluster.
   Response createJob(1: JobConfiguration description, 3: Lock lock, 2: SessionKey session)
 
+  // Enters a job into the cron schedule, without actually starting the job.
+  // If the job is already present in the schedule, this will update the schedule
+  // entry with the new configuration.
+  Response scheduleCronJob(1: JobConfiguration description, 3: Lock lock, 2: SessionKey session)
+
+  // Removes a job from the cron schedule. The request will be denied if the
+  // job was not previously scheduled with scheduleCronJob.
+  Response descheduleCronJob(4: JobKey job, 3: Lock lock, 2: SessionKey session)
+
   // Starts a cron job immediately.  The request will be denied if the specified job does not
   // exist for the role account, or the job is not a cron job.
   Response startCronJob(4: JobKey job, 3: SessionKey session)

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8f3b2040/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java b/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
index 2142f11..4ed655d 100644
--- a/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
@@ -719,6 +719,44 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
   }
 
   @Test
+  public void testScheduleCronJob() throws Exception {
+    expectAuth(ROLE, true);
+    lockManager.validateIfLocked(LOCK_KEY, Optional.<ILock>absent());
+    expect(cronJobManager.hasJob(JOB_KEY)).andReturn(false);
+    cronJobManager.createJob(anyObject(SanitizedCronJob.class));
+    control.replay();
+    assertResponse(OK, thrift.scheduleCronJob(CRON_JOB, DEFAULT_LOCK, SESSION));
+  }
+
+  @Test
+  public void testUpdateScheduledCronJob() throws Exception {
+    expectAuth(ROLE, true);
+    lockManager.validateIfLocked(LOCK_KEY, Optional.<ILock>absent());
+    expect(cronJobManager.hasJob(JOB_KEY)).andReturn(true);
+    cronJobManager.updateJob(anyObject(SanitizedCronJob.class));
+    control.replay();
+    assertResponse(OK, thrift.scheduleCronJob(CRON_JOB, DEFAULT_LOCK, SESSION));
+  }
+
+  @Test
+  public void testDescheduleCronJob() throws Exception {
+    lockManager.validateIfLocked(LOCK_KEY, Optional.<ILock>absent());
+    expect(cronJobManager.deleteJob(JOB_KEY)).andReturn(true);
+    control.replay();
+    assertResponse(OK, thrift.descheduleCronJob(CRON_JOB.getKey(), DEFAULT_LOCK, SESSION));
+  }
+
+  @Test
+  public void testDescheduleCronJobWithError() throws Exception {
+    lockManager.validateIfLocked(LOCK_KEY, Optional.<ILock>absent());
+    expect(cronJobManager.deleteJob(JOB_KEY)).andReturn(false);
+    control.replay();
+    assertResponse(INVALID_REQUEST,
+        thrift.descheduleCronJob(CRON_JOB.getKey(), DEFAULT_LOCK, SESSION));
+  }
+
+
+  @Test
   public void testRewriteShardTaskMissing() throws Exception {
     InstanceKey instance = new InstanceKey(JobKeys.from("foo", "bar", "baz").newBuilder(), 0);
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8f3b2040/src/test/java/org/apache/aurora/scheduler/thrift/aop/ForwardingThrift.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/thrift/aop/ForwardingThrift.java b/src/test/java/org/apache/aurora/scheduler/thrift/aop/ForwardingThrift.java
index 933a56b..b7fef0c 100644
--- a/src/test/java/org/apache/aurora/scheduler/thrift/aop/ForwardingThrift.java
+++ b/src/test/java/org/apache/aurora/scheduler/thrift/aop/ForwardingThrift.java
@@ -123,7 +123,20 @@ abstract class ForwardingThrift implements AuroraAdmin.Iface {
     return delegate.createJob(description, lock, session);
   }
 
-  @Override
+    @Override
+    public Response scheduleCronJob(JobConfiguration description, Lock lock, SessionKey session)
+        throws TException {
+
+      return delegate.scheduleCronJob(description, lock, session);
+    }
+
+    @Override
+    public Response descheduleCronJob(JobKey job, Lock lock, SessionKey session) throws TException {
+
+      return delegate.descheduleCronJob(job, lock, session);
+    }
+
+    @Override
   public Response replaceCronTemplate(JobConfiguration config, Lock lock, SessionKey session)
       throws TException {
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8f3b2040/src/test/python/apache/aurora/client/api/test_scheduler_client.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/api/test_scheduler_client.py b/src/test/python/apache/aurora/client/api/test_scheduler_client.py
index 7bf2c32..e29bae9 100644
--- a/src/test/python/apache/aurora/client/api/test_scheduler_client.py
+++ b/src/test/python/apache/aurora/client/api/test_scheduler_client.py
@@ -101,6 +101,16 @@ class TestSchedulerProxyInjection(unittest.TestCase):
 
     self.make_scheduler_proxy().replaceCronTemplate(JobConfiguration(), Lock())
 
+  def test_scheduleCronJob(self):
+    self.mock_thrift_client.scheduleCronJob(IsA(JobConfiguration), IsA(SessionKey))
+    self.mox.ReplayAll()
+    self.make_scheduler_proxy().scheduleCronJob(JobConfiguration())
+
+  def test_descheduleCronJob(self):
+    self.mock_thrift_client.descheduleCronJob(IsA(JobKey), IsA(SessionKey))
+    self.mox.ReplayAll()
+    self.make_scheduler_proxy().descheduleCronJob(JOB_KEY)
+
   def test_populateJobConfig(self):
     self.mock_thrift_client.populateJobConfig(IsA(JobConfiguration))