You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by ma...@apache.org on 2014/11/27 02:23:24 UTC

incubator-aurora git commit: Adding quota check into replaceCronTemplate rpc.

Repository: incubator-aurora
Updated Branches:
  refs/heads/master 8d45482e3 -> 88baad6a0


Adding quota check into replaceCronTemplate rpc.

Bugs closed: AURORA-946

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


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

Branch: refs/heads/master
Commit: 88baad6a07c30d7abaab5777210efbbb7f020ba4
Parents: 8d45482
Author: Maxim Khutornenko <ma...@apache.org>
Authored: Wed Nov 26 16:47:54 2014 -0800
Committer: Maxim Khutornenko <ma...@apache.org>
Committed: Wed Nov 26 16:47:54 2014 -0800

----------------------------------------------------------------------
 .../thrift/org/apache/aurora/gen/api.thrift     |  1 +
 .../thrift/SchedulerThriftInterface.java        | 63 ++++++++------------
 .../thrift/SchedulerThriftInterfaceTest.java    | 16 +++++
 3 files changed, 41 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/88baad6a/api/src/main/thrift/org/apache/aurora/gen/api.thrift
----------------------------------------------------------------------
diff --git a/api/src/main/thrift/org/apache/aurora/gen/api.thrift b/api/src/main/thrift/org/apache/aurora/gen/api.thrift
index b91fca9..6b63f04 100644
--- a/api/src/main/thrift/org/apache/aurora/gen/api.thrift
+++ b/api/src/main/thrift/org/apache/aurora/gen/api.thrift
@@ -1000,6 +1000,7 @@ service AuroraSchedulerManager extends ReadOnlyScheduler {
   /** Releases the lock acquired earlier in acquireLock call. */
   Response releaseLock(1: Lock lock, 2: LockValidation validation, 3: SessionKey session)
 
+  // TODO(maxim): reevaluate if it's still needed when client updater is gone (AURORA-785).
   /**
    * Replaces the template (configuration) for the existing cron job.
    * The cron job template (configuration) must exist for the call to succeed.

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/88baad6a/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 9087eb2..0898c62 100644
--- a/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java
+++ b/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java
@@ -331,11 +331,11 @@ class SchedulerThriftInterface implements AuroraAdmin.Iface {
     });
   }
 
-  @Override
-  public Response scheduleCronJob(
+  private Response createOrUpdateCronTemplate(
       JobConfiguration mutableJob,
       @Nullable Lock mutableLock,
-      SessionKey session) {
+      SessionKey session,
+      boolean updateOnly) {
 
     IJobConfiguration job = IJobConfiguration.build(mutableJob);
     IJobKey jobKey = JobKeys.assertValid(job.getKey());
@@ -343,11 +343,7 @@ class SchedulerThriftInterface implements AuroraAdmin.Iface {
 
     try {
       sessionValidator.checkAuthenticated(session, ImmutableSet.of(jobKey.getRole()));
-    } catch (AuthFailedException e) {
-      return errorResponse(AUTH_FAILED, e);
-    }
 
-    try {
       SanitizedConfiguration sanitized = SanitizedConfiguration.fromUnsanitized(job);
 
       lockManager.validateIfLocked(
@@ -364,7 +360,7 @@ class SchedulerThriftInterface implements AuroraAdmin.Iface {
       validateTaskLimits(template, count, quotaManager.checkInstanceAddition(template, count));
 
       // TODO(mchucarroll): Merge CronJobManager.createJob/updateJob
-      if (cronJobManager.hasJob(sanitized.getJobConfig().getKey())) {
+      if (updateOnly || cronJobManager.hasJob(sanitized.getJobConfig().getKey())) {
         // The job already has a schedule: so update it.
         cronJobManager.updateJob(SanitizedCronJob.from(sanitized));
       } else {
@@ -372,6 +368,8 @@ class SchedulerThriftInterface implements AuroraAdmin.Iface {
       }
 
       return okEmptyResponse();
+    } catch (AuthFailedException e) {
+      return errorResponse(AUTH_FAILED, e);
     } catch (LockException e) {
       return errorResponse(LOCK_ERROR, e);
     } catch (TaskDescriptionException | TaskValidationException | CronException e) {
@@ -380,6 +378,24 @@ class SchedulerThriftInterface implements AuroraAdmin.Iface {
   }
 
   @Override
+  public Response scheduleCronJob(
+      JobConfiguration mutableJob,
+      @Nullable Lock mutableLock,
+      SessionKey session) {
+
+   return createOrUpdateCronTemplate(mutableJob, mutableLock, session, false);
+  }
+
+  @Override
+  public Response replaceCronTemplate(
+      JobConfiguration mutableJob,
+      @Nullable Lock mutableLock,
+      SessionKey session) {
+
+    return createOrUpdateCronTemplate(mutableJob, mutableLock, session, true);
+  }
+
+  @Override
   public Response descheduleCronJob(
       JobKey mutableJobKey,
       @Nullable Lock mutableLock,
@@ -405,37 +421,6 @@ class SchedulerThriftInterface implements AuroraAdmin.Iface {
   }
 
   @Override
-  public Response replaceCronTemplate(
-      JobConfiguration mutableConfig,
-      @Nullable Lock mutableLock,
-      SessionKey session) {
-
-    requireNonNull(mutableConfig);
-    IJobConfiguration job = IJobConfiguration.build(mutableConfig);
-    IJobKey jobKey = JobKeys.assertValid(job.getKey());
-    requireNonNull(session);
-
-    try {
-      sessionValidator.checkAuthenticated(session, ImmutableSet.of(jobKey.getRole()));
-    } catch (AuthFailedException e) {
-      return errorResponse(AUTH_FAILED, e);
-    }
-
-    try {
-      lockManager.validateIfLocked(
-          ILockKey.build(LockKey.job(jobKey.newBuilder())),
-          Optional.fromNullable(mutableLock).transform(ILock.FROM_BUILDER));
-
-      cronJobManager.updateJob(SanitizedCronJob.fromUnsanitized(job));
-      return okEmptyResponse();
-    } catch (LockException e) {
-      return errorResponse(LOCK_ERROR, e);
-    } catch (CronException | TaskDescriptionException e) {
-      return errorResponse(INVALID_REQUEST, e);
-    }
-  }
-
-  @Override
   public Response populateJobConfig(JobConfiguration description) {
     requireNonNull(description);
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/88baad6a/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 c602b30..b28b3ae 100644
--- a/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
@@ -1189,6 +1189,14 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
   public void testReplaceCronTemplate() throws Exception {
     expectAuth(ROLE, true);
     lockManager.validateIfLocked(LOCK_KEY, Optional.<ILock>absent());
+    SanitizedConfiguration sanitized =
+        SanitizedConfiguration.fromUnsanitized(IJobConfiguration.build(CRON_JOB));
+
+    expect(taskIdGenerator.generate(sanitized.getJobConfig().getTaskConfig(), 1))
+        .andReturn(TASK_ID);
+    expect(quotaManager.checkInstanceAddition(sanitized.getJobConfig().getTaskConfig(), 1))
+        .andReturn(ENOUGH_QUOTA);
+
     cronJobManager.updateJob(anyObject(SanitizedCronJob.class));
     control.replay();
 
@@ -1221,6 +1229,14 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
   public void testReplaceCronTemplateDoesNotExist() throws Exception {
     expectAuth(ROLE, true);
     lockManager.validateIfLocked(LOCK_KEY, Optional.<ILock>absent());
+    SanitizedConfiguration sanitized =
+        SanitizedConfiguration.fromUnsanitized(IJobConfiguration.build(CRON_JOB));
+
+    expect(taskIdGenerator.generate(sanitized.getJobConfig().getTaskConfig(), 1))
+        .andReturn(TASK_ID);
+    expect(quotaManager.checkInstanceAddition(sanitized.getJobConfig().getTaskConfig(), 1))
+        .andReturn(ENOUGH_QUOTA);
+
     cronJobManager.updateJob(anyObject(SanitizedCronJob.class));
     expectLastCall().andThrow(new CronException("Nope"));