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 89E4417B25 for ; Tue, 30 Sep 2014 22:09:48 +0000 (UTC) Received: (qmail 77834 invoked by uid 500); 30 Sep 2014 22:09:48 -0000 Delivered-To: apmail-aurora-commits-archive@aurora.apache.org Received: (qmail 77803 invoked by uid 500); 30 Sep 2014 22:09:48 -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 77794 invoked by uid 99); 30 Sep 2014 22:09:48 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 30 Sep 2014 22:09:48 +0000 X-ASF-Spam-Status: No, hits=-2000.6 required=5.0 tests=ALL_TRUSTED,RP_MATCHES_RCVD 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, 30 Sep 2014 22:09:24 +0000 Received: (qmail 76031 invoked by uid 99); 30 Sep 2014 22:09:21 -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, 30 Sep 2014 22:09:21 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 2D6DCA0C8DE; Tue, 30 Sep 2014 22:09:21 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: wfarner@apache.org To: commits@aurora.incubator.apache.org Message-Id: <51096c8838b94ec7bdcafce51ac68980@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: git commit: Perform job update no-op detection in SchedulerThriftInterface. Date: Tue, 30 Sep 2014 22:09:21 +0000 (UTC) X-Virus-Checked: Checked by ClamAV on apache.org Repository: incubator-aurora Updated Branches: refs/heads/master 0454e85bd -> cdedd20b1 Perform job update no-op detection in SchedulerThriftInterface. Bugs closed: AURORA-769 Reviewed at https://reviews.apache.org/r/26158/ Project: http://git-wip-us.apache.org/repos/asf/incubator-aurora/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-aurora/commit/cdedd20b Tree: http://git-wip-us.apache.org/repos/asf/incubator-aurora/tree/cdedd20b Diff: http://git-wip-us.apache.org/repos/asf/incubator-aurora/diff/cdedd20b Branch: refs/heads/master Commit: cdedd20b1a2087eb918662224a79f99350247798 Parents: 0454e85 Author: Bill Farner Authored: Tue Sep 30 15:07:57 2014 -0700 Committer: Bill Farner Committed: Tue Sep 30 15:07:57 2014 -0700 ---------------------------------------------------------------------- .../thrift/SchedulerThriftInterface.java | 33 ++++++-------- .../aurora/scheduler/updater/JobDiff.java | 47 ++++---------------- .../scheduler/updater/JobUpdateController.java | 13 +----- .../updater/JobUpdateControllerImpl.java | 6 ++- .../thrift/SchedulerThriftInterfaceTest.java | 10 +---- .../aurora/scheduler/updater/JobUpdaterIT.java | 7 ++- 6 files changed, 33 insertions(+), 83 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/cdedd20b/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 2847bdb..01fc345 100644 --- a/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java +++ b/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java @@ -151,7 +151,6 @@ import org.apache.aurora.scheduler.storage.entities.IJobConfiguration; import org.apache.aurora.scheduler.storage.entities.IJobKey; import org.apache.aurora.scheduler.storage.entities.IJobUpdate; import org.apache.aurora.scheduler.storage.entities.IJobUpdateDetails; -import org.apache.aurora.scheduler.storage.entities.IJobUpdateInstructions; import org.apache.aurora.scheduler.storage.entities.IJobUpdateQuery; import org.apache.aurora.scheduler.storage.entities.IJobUpdateRequest; import org.apache.aurora.scheduler.storage.entities.IJobUpdateSettings; @@ -189,7 +188,6 @@ import static org.apache.aurora.scheduler.base.Tasks.ACTIVE_STATES; import static org.apache.aurora.scheduler.quota.QuotaCheckResult.Result.INSUFFICIENT_QUOTA; import static org.apache.aurora.scheduler.thrift.Util.addMessage; import static org.apache.aurora.scheduler.thrift.Util.emptyResponse; -import static org.apache.aurora.scheduler.updater.JobUpdateController.NoopUpdateStateException; /** * Aurora scheduler thrift server implementation. @@ -1354,15 +1352,6 @@ class SchedulerThriftInterface implements AuroraAdmin.Iface { return builder.build(); } - private static Optional buildDesiredState(JobDiff diff, ITaskConfig config) { - Set desiredInstances = diff.getReplacementInstances(); - return desiredInstances.isEmpty() - ? Optional.absent() - : Optional.of(new InstanceTaskConfig() - .setTask(config.newBuilder()) - .setInstances(convertRanges(Numbers.toRanges(desiredInstances)))); - } - @Override public Response startJobUpdate(JobUpdateRequest mutableRequest, SessionKey session) { if (!isUpdaterEnabled) { @@ -1425,6 +1414,10 @@ class SchedulerThriftInterface implements AuroraAdmin.Iface { JobDiff.asMap(request.getTaskConfig(), request.getInstanceCount()), settings.getUpdateOnlyTheseInstances()); + if (diff.isNoop()) { + return addMessage(emptyResponse(), OK, "Job is unchanged by proposed update."); + } + Set invalidScope = diff.getOutOfScopeInstances( Numbers.rangesToInstanceIds(settings.getUpdateOnlyTheseInstances())); if (!invalidScope.isEmpty()) { @@ -1433,18 +1426,22 @@ class SchedulerThriftInterface implements AuroraAdmin.Iface { + invalidScope); } - IJobUpdateInstructions instructions = - IJobUpdateInstructions.build(new JobUpdateInstructions() - .setSettings(settings.newBuilder()) - .setInitialState(buildInitialState(diff.getReplacedInstances())) - .setDesiredState(buildDesiredState(diff, request.getTaskConfig()).orNull())); + JobUpdateInstructions instructions = new JobUpdateInstructions() + .setSettings(settings.newBuilder()) + .setInitialState(buildInitialState(diff.getReplacedInstances())); + if (!diff.getReplacementInstances().isEmpty()) { + instructions.setDesiredState( + new InstanceTaskConfig() + .setTask(request.getTaskConfig().newBuilder()) + .setInstances(convertRanges(Numbers.toRanges(diff.getReplacementInstances())))); + } IJobUpdate update = IJobUpdate.build(new JobUpdate() .setSummary(new JobUpdateSummary() .setJobKey(job.newBuilder()) .setUpdateId(updateId) .setUser(context.getIdentity())) - .setInstructions(instructions.newBuilder())); + .setInstructions(instructions)); try { validateTaskLimits( request.getTaskConfig(), @@ -1453,8 +1450,6 @@ class SchedulerThriftInterface implements AuroraAdmin.Iface { jobUpdateController.start(update, context.getIdentity()); return okResponse(Result.startJobUpdateResult(new StartJobUpdateResult(updateId))); - } catch (NoopUpdateStateException e) { - return addMessage(emptyResponse(), OK, e.getMessage()); } catch (UpdateStateException | TaskValidationException e) { return errorResponse(INVALID_REQUEST, e); } http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/cdedd20b/src/main/java/org/apache/aurora/scheduler/updater/JobDiff.java ---------------------------------------------------------------------- diff --git a/src/main/java/org/apache/aurora/scheduler/updater/JobDiff.java b/src/main/java/org/apache/aurora/scheduler/updater/JobDiff.java index 5daefff..88a25e6 100644 --- a/src/main/java/org/apache/aurora/scheduler/updater/JobDiff.java +++ b/src/main/java/org/apache/aurora/scheduler/updater/JobDiff.java @@ -33,9 +33,7 @@ import org.apache.aurora.scheduler.base.Numbers; import org.apache.aurora.scheduler.base.Query; import org.apache.aurora.scheduler.base.Tasks; import org.apache.aurora.scheduler.storage.TaskStore; -import org.apache.aurora.scheduler.storage.entities.IInstanceTaskConfig; import org.apache.aurora.scheduler.storage.entities.IJobKey; -import org.apache.aurora.scheduler.storage.entities.IJobUpdateInstructions; import org.apache.aurora.scheduler.storage.entities.IRange; import org.apache.aurora.scheduler.storage.entities.ITaskConfig; @@ -81,6 +79,15 @@ public final class JobDiff { return ImmutableSet.copyOf(Sets.difference(scope, allAlteredInstances)); } + /** + * Checks whether this diff contains no work to be done. + * + * @return {@code true} if this diff is a no-op, otherwise {@code false}. + */ + public boolean isNoop() { + return replacedInstances.isEmpty() && replacementInstances.isEmpty(); + } + @Override public int hashCode() { return Objects.hash(replacedInstances, replacementInstances); @@ -186,40 +193,4 @@ public final class JobDiff { DiscreteDomain.integers()); return ImmutableMap.copyOf(Maps.asMap(desiredInstances, Functions.constant(config))); } - - /** - * Identical to {@link #asMap(ITaskConfig, int)}, extracting fields from {@code config}. - * - * @param config Instance task configuration to represent as an instance mapping. - * @return An instance ID to task config mapping that is equivalent to {@code config}. - */ - private static Map asMap(IInstanceTaskConfig config) { - return Maps.asMap( - Numbers.rangesToInstanceIds(config.getInstances()), - Functions.constant(config.getTask())); - } - - /** - * Determines whether a job update has no work to do. - * - * @param taskStore Store to fetch job state from. - * @param job Job being updated. - * @param instructions Job update instructions. - * @return {@code true} if the update would result in zero instances being modified, - * otherwise {@code false}. - */ - public static boolean isNoopUpdate( - TaskStore taskStore, - IJobKey job, - IJobUpdateInstructions instructions) { - - JobDiff diff = JobDiff.compute( - taskStore, - job, - instructions.isSetDesiredState() - ? ImmutableMap.copyOf(asMap(instructions.getDesiredState())) - : ImmutableMap.of(), - instructions.getSettings().getUpdateOnlyTheseInstances()); - return diff.getReplacedInstances().isEmpty() && diff.getReplacementInstances().isEmpty(); - } } http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/cdedd20b/src/main/java/org/apache/aurora/scheduler/updater/JobUpdateController.java ---------------------------------------------------------------------- diff --git a/src/main/java/org/apache/aurora/scheduler/updater/JobUpdateController.java b/src/main/java/org/apache/aurora/scheduler/updater/JobUpdateController.java index f37fbef..b8dafe0 100644 --- a/src/main/java/org/apache/aurora/scheduler/updater/JobUpdateController.java +++ b/src/main/java/org/apache/aurora/scheduler/updater/JobUpdateController.java @@ -38,13 +38,11 @@ public interface JobUpdateController { * * @param update Instructions for what job to update, and how to update it. * @param updatingUser User initiating the update. - * @throws NoopUpdateStateException If the update does not change the job. * @throws UpdateStateException If the update cannot be started, for example if the instructions * are invalid, or if there is already an in-progress update for the * job. */ - void start(IJobUpdate update, String updatingUser) - throws NoopUpdateStateException, UpdateStateException; + void start(IJobUpdate update, String updatingUser) throws UpdateStateException; /** * Pauses an in-progress update. @@ -100,13 +98,4 @@ public interface JobUpdateController { * updates, but resumes after a restart of the scheduler process. */ void systemResume(); - - /** - * Thrown when an update was started that requires no work. - */ - class NoopUpdateStateException extends UpdateStateException { - public NoopUpdateStateException() { - super("Job is unchanged by proposed update."); - } - } } http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/cdedd20b/src/main/java/org/apache/aurora/scheduler/updater/JobUpdateControllerImpl.java ---------------------------------------------------------------------- diff --git a/src/main/java/org/apache/aurora/scheduler/updater/JobUpdateControllerImpl.java b/src/main/java/org/apache/aurora/scheduler/updater/JobUpdateControllerImpl.java index 97bc1df..faa2136 100644 --- a/src/main/java/org/apache/aurora/scheduler/updater/JobUpdateControllerImpl.java +++ b/src/main/java/org/apache/aurora/scheduler/updater/JobUpdateControllerImpl.java @@ -55,6 +55,7 @@ import org.apache.aurora.scheduler.storage.entities.IJobInstanceUpdateEvent; import org.apache.aurora.scheduler.storage.entities.IJobKey; import org.apache.aurora.scheduler.storage.entities.IJobUpdate; import org.apache.aurora.scheduler.storage.entities.IJobUpdateEvent; +import org.apache.aurora.scheduler.storage.entities.IJobUpdateInstructions; import org.apache.aurora.scheduler.storage.entities.IJobUpdateQuery; import org.apache.aurora.scheduler.storage.entities.IJobUpdateSummary; import org.apache.aurora.scheduler.storage.entities.ILock; @@ -134,13 +135,14 @@ class JobUpdateControllerImpl implements JobUpdateController { throws UpdateStateException { IJobUpdateSummary summary = update.getSummary(); + IJobUpdateInstructions instructions = update.getInstructions(); IJobKey job = summary.getJobKey(); // Validate the update configuration by making sure we can create an updater for it. updateFactory.newUpdate(update.getInstructions(), true); - if (JobDiff.isNoopUpdate(storeProvider.getTaskStore(), job, update.getInstructions())) { - throw new NoopUpdateStateException(); + if (instructions.getInitialState().isEmpty() && !instructions.isSetDesiredState()) { + throw new IllegalArgumentException("Update instruction is a no-op."); } LOG.info("Starting update for job " + job); http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/cdedd20b/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 12b19fd..60c1582 100644 --- a/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java +++ b/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java @@ -167,7 +167,6 @@ import static org.apache.aurora.scheduler.thrift.SchedulerThriftInterface.MAX_TA import static org.apache.aurora.scheduler.thrift.SchedulerThriftInterface.killedByMessage; import static org.apache.aurora.scheduler.thrift.SchedulerThriftInterface.restartedByMessage; import static org.apache.aurora.scheduler.thrift.SchedulerThriftInterface.transitionMessage; -import static org.apache.aurora.scheduler.updater.JobUpdateController.NoopUpdateStateException; import static org.easymock.EasyMock.anyObject; import static org.easymock.EasyMock.eq; import static org.easymock.EasyMock.expect; @@ -2288,10 +2287,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest { expectAuth(ROLE, true); expect(cronJobManager.hasJob(JOB_KEY)).andReturn(false); expect(uuidGenerator.createNew()).andReturn(UU_ID); - ITaskConfig newTask = buildScheduledTask(0, 8).getAssignedTask().getTask(); - expect(taskIdGenerator.generate(newTask, 1)).andReturn(TASK_ID); - expect(quotaManager.checkQuota(ROLE, IResourceAggregate.build( - new ResourceAggregate(1, 8, 1024)))).andReturn(ENOUGH_QUOTA); + ITaskConfig newTask = buildScheduledTask(0, 5).getAssignedTask().getTask(); IScheduledTask oldTask = buildScheduledTask(0, 5); storageUtil.expectTaskFetch(Query.unscoped().byJob(JOB_KEY).active(), oldTask); @@ -2300,8 +2296,6 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest { 1, newTask, ImmutableMap.of(oldTask.getAssignedTask().getTask(), ImmutableSet.of(new Range(0, 0)))); - jobUpdateController.start(update, USER); - expectLastCall().andThrow(new NoopUpdateStateException()); control.replay(); JobUpdateRequest request = buildJobUpdateRequest(update); @@ -2328,7 +2322,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest { control.replay(); JobUpdateRequest request = buildJobUpdateRequest(IJobUpdate.build(builder)); - assertResponse(INVALID_REQUEST, thrift.startJobUpdate(request, SESSION)); + assertResponse(OK, thrift.startJobUpdate(request, SESSION)); } @Test http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/cdedd20b/src/test/java/org/apache/aurora/scheduler/updater/JobUpdaterIT.java ---------------------------------------------------------------------- diff --git a/src/test/java/org/apache/aurora/scheduler/updater/JobUpdaterIT.java b/src/test/java/org/apache/aurora/scheduler/updater/JobUpdaterIT.java index 7254588..8843990 100644 --- a/src/test/java/org/apache/aurora/scheduler/updater/JobUpdaterIT.java +++ b/src/test/java/org/apache/aurora/scheduler/updater/JobUpdaterIT.java @@ -118,7 +118,6 @@ import static org.apache.aurora.gen.ScheduleStatus.RUNNING; import static org.apache.aurora.gen.ScheduleStatus.STARTING; import static org.apache.aurora.scheduler.events.PubsubEvent.SchedulerActive; import static org.apache.aurora.scheduler.storage.Storage.MutateWork.NoResult; -import static org.apache.aurora.scheduler.updater.JobUpdateController.NoopUpdateStateException; import static org.apache.aurora.scheduler.updater.UpdateFactory.UpdateFactoryImpl.expandInstanceIds; import static org.easymock.EasyMock.expectLastCall; import static org.junit.Assert.assertEquals; @@ -798,8 +797,8 @@ public class JobUpdaterIT extends EasyMockTest { assertState(ERROR, ImmutableMultimap.of()); } - @Test(expected = NoopUpdateStateException.class) - public void testNoopUpdate() throws Exception { + @Test + public void testImmediatelySuccessfulUpdate() throws Exception { control.replay(); final IJobUpdate update = makeJobUpdate(makeInstanceConfig(0, 2, NEW_CONFIG)); @@ -811,7 +810,7 @@ public class JobUpdaterIT extends EasyMockTest { updater.start(update, USER); } - @Test(expected = NoopUpdateStateException.class) + @Test(expected = IllegalArgumentException.class) public void testNoopUpdateEmptyDiff() throws Exception { control.replay();