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 4309317C85 for ; Tue, 10 Mar 2015 18:02:00 +0000 (UTC) Received: (qmail 44599 invoked by uid 500); 10 Mar 2015 18:02:00 -0000 Delivered-To: apmail-aurora-commits-archive@aurora.apache.org Received: (qmail 44571 invoked by uid 500); 10 Mar 2015 18:02:00 -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 44561 invoked by uid 99); 10 Mar 2015 18:02:00 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 10 Mar 2015 18:02:00 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED,T_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, 10 Mar 2015 18:01:34 +0000 Received: (qmail 42542 invoked by uid 99); 10 Mar 2015 18:01:31 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 10 Mar 2015 18:01:31 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 0BB14E1862; Tue, 10 Mar 2015 18:01:31 +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: <45aedb2f206045fa82dc452d7f2bfea5@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: incubator-aurora git commit: Include messages with internal job updater state transitions. Date: Tue, 10 Mar 2015 18:01:31 +0000 (UTC) X-Virus-Checked: Checked by ClamAV on apache.org Repository: incubator-aurora Updated Branches: refs/heads/master 1b1931cc9 -> 48dc4c7ad Include messages with internal job updater state transitions. Bugs closed: AURORA-1077 Reviewed at https://reviews.apache.org/r/31814/ Project: http://git-wip-us.apache.org/repos/asf/incubator-aurora/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-aurora/commit/48dc4c7a Tree: http://git-wip-us.apache.org/repos/asf/incubator-aurora/tree/48dc4c7a Diff: http://git-wip-us.apache.org/repos/asf/incubator-aurora/diff/48dc4c7a Branch: refs/heads/master Commit: 48dc4c7ad28eeea3e7108c1c0b408f4010a27aa1 Parents: 1b1931c Author: Bill Farner Authored: Tue Mar 10 11:00:18 2015 -0700 Committer: Bill Farner Committed: Tue Mar 10 11:00:18 2015 -0700 ---------------------------------------------------------------------- .../scheduler/updater/InstanceUpdater.java | 14 +-- .../updater/JobUpdateControllerImpl.java | 92 +++++++++++++------- .../scheduler/updater/OneWayJobUpdater.java | 19 ++-- .../aurora/scheduler/updater/SideEffect.java | 20 ++++- .../scheduler/updater/StateEvaluator.java | 49 ++++++++--- .../scheduler/updater/InstanceUpdaterTest.java | 6 +- .../aurora/scheduler/updater/JobUpdaterIT.java | 24 +++-- .../scheduler/updater/OneWayJobUpdaterTest.java | 11 ++- 8 files changed, 165 insertions(+), 70 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/48dc4c7a/src/main/java/org/apache/aurora/scheduler/updater/InstanceUpdater.java ---------------------------------------------------------------------- diff --git a/src/main/java/org/apache/aurora/scheduler/updater/InstanceUpdater.java b/src/main/java/org/apache/aurora/scheduler/updater/InstanceUpdater.java index 09f2a74..14753cf 100644 --- a/src/main/java/org/apache/aurora/scheduler/updater/InstanceUpdater.java +++ b/src/main/java/org/apache/aurora/scheduler/updater/InstanceUpdater.java @@ -36,7 +36,8 @@ import static org.apache.aurora.gen.ScheduleStatus.KILLING; import static org.apache.aurora.gen.ScheduleStatus.RUNNING; import static org.apache.aurora.scheduler.updater.StateEvaluator.Result.EVALUATE_AFTER_MIN_RUNNING_MS; import static org.apache.aurora.scheduler.updater.StateEvaluator.Result.EVALUATE_ON_STATE_CHANGE; -import static org.apache.aurora.scheduler.updater.StateEvaluator.Result.FAILED; +import static org.apache.aurora.scheduler.updater.StateEvaluator.Result.FAILED_STUCK; +import static org.apache.aurora.scheduler.updater.StateEvaluator.Result.FAILED_TERMINATED; import static org.apache.aurora.scheduler.updater.StateEvaluator.Result.KILL_TASK_AND_EVALUATE_ON_STATE_CHANGE; import static org.apache.aurora.scheduler.updater.StateEvaluator.Result.REPLACE_TASK_AND_EVALUATE_ON_STATE_CHANGE; import static org.apache.aurora.scheduler.updater.StateEvaluator.Result.SUCCEEDED; @@ -131,10 +132,10 @@ class InstanceUpdater implements StateEvaluator> { } } - private StateEvaluator.Result addFailureAndCheckIfFailed() { + private boolean addFailureAndCheckIfFailed() { LOG.info("Observed updated task failure."); observedFailures++; - return observedFailures > toleratedFailures ? FAILED : EVALUATE_ON_STATE_CHANGE; + return observedFailures > toleratedFailures; } private StateEvaluator.Result handleActualAndDesiredPresent(IScheduledTask actualState) { @@ -156,14 +157,13 @@ class InstanceUpdater implements StateEvaluator> { } else if (Tasks.isTerminated(status)) { // The desired task has terminated, this is a failure. LOG.info("Task is in terminal state " + status); - return addFailureAndCheckIfFailed(); + return addFailureAndCheckIfFailed() ? FAILED_TERMINATED : EVALUATE_ON_STATE_CHANGE; } else if (appearsStuck(actualState)) { LOG.info("Task appears stuck."); // The task is not running, but not terminated, and appears to have been in this state // long enough that we should intervene. - StateEvaluator.Result updaterStatus = addFailureAndCheckIfFailed(); - return (updaterStatus == FAILED) - ? updaterStatus + return addFailureAndCheckIfFailed() + ? FAILED_STUCK : KILL_TASK_AND_EVALUATE_ON_STATE_CHANGE; } else { // The task is in a transient state on the way into or out of running, check back later. http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/48dc4c7a/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 5d3b16a..5882488 100644 --- a/src/main/java/org/apache/aurora/scheduler/updater/JobUpdateControllerImpl.java +++ b/src/main/java/org/apache/aurora/scheduler/updater/JobUpdateControllerImpl.java @@ -65,6 +65,7 @@ import org.apache.aurora.scheduler.storage.entities.IJobUpdateSummary; import org.apache.aurora.scheduler.storage.entities.ILock; import org.apache.aurora.scheduler.storage.entities.ILockKey; import org.apache.aurora.scheduler.storage.entities.IScheduledTask; +import org.apache.aurora.scheduler.updater.StateEvaluator.Failure; import static java.util.Objects.requireNonNull; @@ -98,13 +99,11 @@ import static org.apache.aurora.scheduler.updater.SideEffect.InstanceUpdateStatu /** * Implementation of an updater that orchestrates the process of gradually updating the * configuration of tasks in a job. - * >p> + *

* TODO(wfarner): Consider using AbstractIdleService here. */ class JobUpdateControllerImpl implements JobUpdateController { private static final Logger LOG = Logger.getLogger(JobUpdateControllerImpl.class.getName()); - private static final String INTERNAL_USER = "Aurora Updater"; - private static final Optional NO_USER = Optional.absent(); private final UpdateFactory updateFactory; private final LockManager lockManager; @@ -190,8 +189,7 @@ class JobUpdateControllerImpl implements JobUpdateController { recordAndChangeJobUpdateStatus( storeProvider, summary.getKey(), - status, - Optional.of(updatingUser)); + newEvent(status).setUser(updatingUser)); } }); } @@ -225,7 +223,7 @@ class JobUpdateControllerImpl implements JobUpdateController { : GET_ACTIVE_RESUME_STATE; JobUpdateStatus newStatus = stateChange.apply(update.getSummary().getState().getStatus()); - changeUpdateStatus(storeProvider, update.getSummary(), newStatus, Optional.of(user)); + changeUpdateStatus(storeProvider, update.getSummary(), newEvent(newStatus).setUser(user)); } }); } @@ -261,7 +259,7 @@ class JobUpdateControllerImpl implements JobUpdateController { } try { - changeJobUpdateStatus(storeProvider, key, status, NO_USER, false); + changeJobUpdateStatus(storeProvider, key, newEvent(status), false); } catch (UpdateStateException e) { throw Throwables.propagate(e); } @@ -292,7 +290,7 @@ class JobUpdateControllerImpl implements JobUpdateController { unscopedChangeUpdateStatus( key, GET_UNBLOCKED_STATE, - Optional.of(INTERNAL_USER)); + Optional.absent()); } catch (UpdateStateException e) { LOG.severe("Error while processing job update pulse: " + e); } @@ -385,9 +383,9 @@ class JobUpdateControllerImpl implements JobUpdateController { throw new UpdateStateException("Update does not exist " + key); } - JobUpdateStatus status = update.getState().getStatus(); - JobUpdateStatus newStatus = requireNonNull(stateChange.apply(status)); - changeUpdateStatus(storeProvider, update, newStatus, user); + JobUpdateStatus newStatus = + requireNonNull(stateChange.apply(update.getState().getStatus())); + changeUpdateStatus(storeProvider, update, newEvent(newStatus).setUser(user.orNull())); } }); } @@ -395,24 +393,22 @@ class JobUpdateControllerImpl implements JobUpdateController { private void changeUpdateStatus( MutableStoreProvider storeProvider, IJobUpdateSummary updateSummary, - JobUpdateStatus newStatus, - Optional user) throws UpdateStateException { + JobUpdateEvent event) throws UpdateStateException { - if (updateSummary.getState().getStatus() == newStatus) { + if (updateSummary.getState().getStatus() == event.getStatus()) { return; } - assertTransitionAllowed(updateSummary.getState().getStatus(), newStatus); - recordAndChangeJobUpdateStatus(storeProvider, updateSummary.getKey(), newStatus, user); + assertTransitionAllowed(updateSummary.getState().getStatus(), event.getStatus()); + recordAndChangeJobUpdateStatus(storeProvider, updateSummary.getKey(), event); } private void recordAndChangeJobUpdateStatus( MutableStoreProvider storeProvider, IJobUpdateKey key, - JobUpdateStatus status, - Optional user) throws UpdateStateException { + JobUpdateEvent event) throws UpdateStateException { - changeJobUpdateStatus(storeProvider, key, status, user, true); + changeJobUpdateStatus(storeProvider, key, event, true); } private static final Set TERMINAL_STATES = ImmutableSet.of( @@ -426,8 +422,7 @@ class JobUpdateControllerImpl implements JobUpdateController { private void changeJobUpdateStatus( MutableStoreProvider storeProvider, IJobUpdateKey key, - JobUpdateStatus newStatus, - Optional user, + JobUpdateEvent proposedEvent, boolean recordChange) throws UpdateStateException { JobUpdateStatus status; @@ -436,7 +431,7 @@ class JobUpdateControllerImpl implements JobUpdateController { JobUpdateStore.Mutable updateStore = storeProvider.getJobUpdateStore(); Optional updateLock = updateStore.getLockToken(key); if (updateLock.isPresent()) { - status = newStatus; + status = proposedEvent.getStatus(); record = recordChange; } else { LOG.severe("Update " + key + " does not have a lock"); @@ -448,10 +443,7 @@ class JobUpdateControllerImpl implements JobUpdateController { if (record) { updateStore.saveJobUpdateEvent( key, - IJobUpdateEvent.build(new JobUpdateEvent() - .setStatus(status) - .setUser(user.orNull()) - .setTimestampMs(clock.nowMillis()))); + IJobUpdateEvent.build(proposedEvent.setTimestampMs(clock.nowMillis()).setStatus(status))); } if (TERMINAL_STATES.contains(status)) { @@ -483,7 +475,11 @@ class JobUpdateControllerImpl implements JobUpdateController { update = updateFactory.newUpdate(jobUpdate.getInstructions(), action == ROLL_FORWARD); } catch (RuntimeException e) { LOG.log(Level.WARNING, "Uncaught exception: " + e, e); - changeJobUpdateStatus(storeProvider, key, ERROR, user, true); + changeJobUpdateStatus( + storeProvider, + key, + newEvent(ERROR).setMessage("Internal scheduler error: " + e.getMessage()), + true); return; } updates.put(job, update); @@ -525,6 +521,12 @@ class JobUpdateControllerImpl implements JobUpdateController { } } + @VisibleForTesting + static final String LOST_LOCK_MESSAGE = "Updater has lost its exclusive lock, unable to proceed."; + + @VisibleForTesting + static final String PULSE_TIMEOUT_MESSAGE = "Pulses from external service have timed out."; + private void evaluateUpdater( final MutableStoreProvider storeProvider, final UpdateFactory.Update update, @@ -536,7 +538,10 @@ class JobUpdateControllerImpl implements JobUpdateController { JobUpdateStore.Mutable updateStore = storeProvider.getJobUpdateStore(); if (!updateStore.getLockToken(key).isPresent()) { - recordAndChangeJobUpdateStatus(storeProvider, key, ERROR, NO_USER); + recordAndChangeJobUpdateStatus( + storeProvider, + key, + newEvent(ERROR).setMessage(LOST_LOCK_MESSAGE)); return; } @@ -544,7 +549,10 @@ class JobUpdateControllerImpl implements JobUpdateController { if (isCoordinatedAndPulseExpired(key, instructions)) { // Move coordinated update into awaiting pulse state. JobUpdateStatus blockedStatus = getBlockedState(summary.getState().getStatus()); - changeUpdateStatus(storeProvider, summary, blockedStatus, Optional.of(INTERNAL_USER)); + changeUpdateStatus( + storeProvider, + summary, + newEvent(blockedStatus).setMessage(PULSE_TIMEOUT_MESSAGE)); return; } @@ -600,11 +608,24 @@ class JobUpdateControllerImpl implements JobUpdateController { "A terminal state should not specify actions: " + result); } + JobUpdateEvent event = new JobUpdateEvent(); if (status == SUCCEEDED) { - changeUpdateStatus(storeProvider, summary, update.getSuccessStatus(), NO_USER); + event.setStatus(update.getSuccessStatus()); } else { - changeUpdateStatus(storeProvider, summary, update.getFailureStatus(), NO_USER); + event.setStatus(update.getFailureStatus()); + // Generate a transition message based on one (arbitrary) instance in the group that pushed + // the update over the failure threshold (in all likelihood this group is of size 1). + // This is done as a rough cut to aid in diagnosing a failed update, as generating a + // complete summary would likely be of dubious value. + for (Map.Entry entry : result.getSideEffects().entrySet()) { + Optional failure = entry.getValue().getFailure(); + if (failure.isPresent()) { + event.setMessage(failureMessage(entry.getKey(), failure.get())); + break; + } + } } + changeUpdateStatus(storeProvider, summary, event); } else { LOG.info("Executing side-effects for update of " + key + ": " + result.getSideEffects()); for (Map.Entry entry : result.getSideEffects().entrySet()) { @@ -630,6 +651,11 @@ class JobUpdateControllerImpl implements JobUpdateController { } } + @VisibleForTesting + static String failureMessage(int instanceId, Failure failure) { + return String.format("Latest failure: instance %d %s", instanceId, failure.getReason()); + } + /** * Associates an instance updater state change and the job's update status to an action. */ @@ -660,6 +686,10 @@ class JobUpdateControllerImpl implements JobUpdateController { return IJobUpdateQuery.build(new JobUpdateQuery().setKey(key.newBuilder())); } + private JobUpdateEvent newEvent(JobUpdateStatus status) { + return new JobUpdateEvent().setStatus(status); + } + private Runnable getDeferredEvaluator(final IInstanceKey instance, final IJobUpdateKey key) { return new Runnable() { @Override http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/48dc4c7a/src/main/java/org/apache/aurora/scheduler/updater/OneWayJobUpdater.java ---------------------------------------------------------------------- diff --git a/src/main/java/org/apache/aurora/scheduler/updater/OneWayJobUpdater.java b/src/main/java/org/apache/aurora/scheduler/updater/OneWayJobUpdater.java index 27a5b90..0f4913a 100644 --- a/src/main/java/org/apache/aurora/scheduler/updater/OneWayJobUpdater.java +++ b/src/main/java/org/apache/aurora/scheduler/updater/OneWayJobUpdater.java @@ -249,18 +249,23 @@ class OneWayJobUpdater { } Result result = evaluator.evaluate(actualState); - if (result == Result.SUCCEEDED) { - stateMachine.transition(SUCCEEDED); - statusChanges.add(SUCCEEDED); - } else if (result == Result.FAILED) { - stateMachine.transition(FAILED); - statusChanges.add(FAILED); + if (TERMINAL_RESULT_TO_STATUS.containsKey(result)) { + SideEffect.InstanceUpdateStatus status = TERMINAL_RESULT_TO_STATUS.get(result); + stateMachine.transition(status); + statusChanges.add(status); } - return new SideEffect(result.getAction(), statusChanges.build()); + return new SideEffect(result.getAction(), statusChanges.build(), result.getFailure()); } } + private static final Map TERMINAL_RESULT_TO_STATUS = + ImmutableMap.of( + Result.SUCCEEDED, SUCCEEDED, + Result.FAILED_STUCK, FAILED, + Result.FAILED_TERMINATED, FAILED + ); + /** * Status of the job update. */ http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/48dc4c7a/src/main/java/org/apache/aurora/scheduler/updater/SideEffect.java ---------------------------------------------------------------------- diff --git a/src/main/java/org/apache/aurora/scheduler/updater/SideEffect.java b/src/main/java/org/apache/aurora/scheduler/updater/SideEffect.java index 27e0654..3f49242 100644 --- a/src/main/java/org/apache/aurora/scheduler/updater/SideEffect.java +++ b/src/main/java/org/apache/aurora/scheduler/updater/SideEffect.java @@ -18,6 +18,8 @@ import java.util.Set; import com.google.common.base.Optional; +import org.apache.aurora.scheduler.updater.StateEvaluator.Failure; + import static java.util.Objects.requireNonNull; /** @@ -27,16 +29,23 @@ import static java.util.Objects.requireNonNull; public class SideEffect { private final Optional action; private final Set statusChanges; + private final Optional failure; /** * Creates a new side-effect. * * @param action Action to be taken on the instance, if necessary. * @param statusChanges Any status changes to the instance monitor. + * @param failure The failure encountered, if any. */ - public SideEffect(Optional action, Set statusChanges) { + public SideEffect( + Optional action, + Set statusChanges, + Optional failure) { + this.action = requireNonNull(action); this.statusChanges = requireNonNull(statusChanges); + this.failure = requireNonNull(failure); } /** @@ -58,6 +67,15 @@ public class SideEffect { } /** + * Gets the failure observed during evaluation, if there was one. + * + * @return Instance failure. + */ + public Optional getFailure() { + return failure; + } + + /** * Tests whether any of multiple side-effects contain {@link #getAction() actions} to be * performed. * http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/48dc4c7a/src/main/java/org/apache/aurora/scheduler/updater/StateEvaluator.java ---------------------------------------------------------------------- diff --git a/src/main/java/org/apache/aurora/scheduler/updater/StateEvaluator.java b/src/main/java/org/apache/aurora/scheduler/updater/StateEvaluator.java index 66b0e4b..1590e12 100644 --- a/src/main/java/org/apache/aurora/scheduler/updater/StateEvaluator.java +++ b/src/main/java/org/apache/aurora/scheduler/updater/StateEvaluator.java @@ -13,17 +13,18 @@ */ package org.apache.aurora.scheduler.updater; -import java.util.Objects; - import com.google.common.base.Optional; +import static java.util.Objects.requireNonNull; + /** * Determines actions that must be taken to change the configuration of a running task. *

* A state evaluator is expected to be used multiple times over the course of changing an active - * task's configuration. This should be invoked every time the state of an instance changes, to + * task's configuration. This should be invoked every time the state of an instance changes to * determine what action to take next. It's expected that it will eventually converge by - * {@link Result#SUCCEEDED succeeding} or {@link Result#FAILED failing}. + * {@link Result#SUCCEEDED succeeding}, or failing with {@link Result#FAILED_STUCK} or + * {@link Result#FAILED_TERMINATED}. * * @param Instance state type. */ @@ -47,22 +48,46 @@ interface StateEvaluator { */ Result evaluate(T actualState); + Optional NO_FAILURE = Optional.absent(); + enum Result { - EVALUATE_ON_STATE_CHANGE(Optional.of(InstanceAction.AWAIT_STATE_CHANGE)), - REPLACE_TASK_AND_EVALUATE_ON_STATE_CHANGE(Optional.of(InstanceAction.ADD_TASK)), - KILL_TASK_AND_EVALUATE_ON_STATE_CHANGE(Optional.of(InstanceAction.KILL_TASK)), - EVALUATE_AFTER_MIN_RUNNING_MS(Optional.of(InstanceAction.WATCH_TASK)), - SUCCEEDED(Optional.absent()), - FAILED(Optional.absent()); + EVALUATE_ON_STATE_CHANGE(Optional.of(InstanceAction.AWAIT_STATE_CHANGE), NO_FAILURE), + REPLACE_TASK_AND_EVALUATE_ON_STATE_CHANGE(Optional.of(InstanceAction.ADD_TASK), NO_FAILURE), + KILL_TASK_AND_EVALUATE_ON_STATE_CHANGE(Optional.of(InstanceAction.KILL_TASK), NO_FAILURE), + EVALUATE_AFTER_MIN_RUNNING_MS(Optional.of(InstanceAction.WATCH_TASK), NO_FAILURE), + SUCCEEDED(Optional.absent(), NO_FAILURE), + FAILED_STUCK(Optional.absent(), Optional.of(Failure.STUCK)), + FAILED_TERMINATED(Optional.absent(), Optional.of(Failure.EXITED)); private final Optional action; + private final Optional failure; - Result(Optional action) { - this.action = Objects.requireNonNull(action); + Result(Optional action, Optional failure) { + this.action = requireNonNull(action); + this.failure = requireNonNull(failure); } public Optional getAction() { return action; } + + public Optional getFailure() { + return failure; + } + } + + enum Failure { + STUCK("took too long to transition from a transient state."), + EXITED("exited."); + + private final String reason; + + Failure(String reason) { + this.reason = reason; + } + + public String getReason() { + return reason; + } } } http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/48dc4c7a/src/test/java/org/apache/aurora/scheduler/updater/InstanceUpdaterTest.java ---------------------------------------------------------------------- diff --git a/src/test/java/org/apache/aurora/scheduler/updater/InstanceUpdaterTest.java b/src/test/java/org/apache/aurora/scheduler/updater/InstanceUpdaterTest.java index 4db0080..09c147e 100644 --- a/src/test/java/org/apache/aurora/scheduler/updater/InstanceUpdaterTest.java +++ b/src/test/java/org/apache/aurora/scheduler/updater/InstanceUpdaterTest.java @@ -150,7 +150,7 @@ public class InstanceUpdaterTest { f.evaluate(REPLACE_TASK_AND_EVALUATE_ON_STATE_CHANGE, FINISHED); f.setActualState(NEW); f.evaluate(EVALUATE_AFTER_MIN_RUNNING_MS, PENDING, ASSIGNED, STARTING, RUNNING); - f.evaluate(Result.FAILED, FAILED); + f.evaluate(Result.FAILED_TERMINATED, FAILED); } @Test @@ -206,7 +206,7 @@ public class InstanceUpdaterTest { f.setActualState(NEW); f.evaluate(EVALUATE_AFTER_MIN_RUNNING_MS, PENDING); f.advanceTime(MAX_NON_RUNNING_TIME); - f.evaluateCurrentState(Result.FAILED); + f.evaluateCurrentState(Result.FAILED_STUCK); } @Test @@ -227,7 +227,7 @@ public class InstanceUpdaterTest { f.evaluate(EVALUATE_AFTER_MIN_RUNNING_MS, ASSIGNED, STARTING, RUNNING); f.evaluate(EVALUATE_AFTER_MIN_RUNNING_MS, KILLING); f.advanceTime(MAX_NON_RUNNING_TIME); - f.evaluateCurrentState(Result.FAILED); + f.evaluateCurrentState(Result.FAILED_STUCK); } @Test(expected = IllegalArgumentException.class) http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/48dc4c7a/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 0fe153c..e119c49 100644 --- a/src/test/java/org/apache/aurora/scheduler/updater/JobUpdaterIT.java +++ b/src/test/java/org/apache/aurora/scheduler/updater/JobUpdaterIT.java @@ -98,6 +98,7 @@ import org.apache.aurora.scheduler.storage.mem.MemStorage.Delegated; import org.apache.aurora.scheduler.storage.mem.MemStorageModule; import org.apache.aurora.scheduler.testing.FakeScheduledExecutor; import org.apache.aurora.scheduler.testing.FakeStatsProvider; +import org.apache.aurora.scheduler.updater.StateEvaluator.Failure; import org.easymock.EasyMock; import org.easymock.IExpectationSetters; import org.junit.After; @@ -273,16 +274,25 @@ public class JobUpdaterIT extends EasyMockTest { } }; - private void assertState( - JobUpdateStatus expected, - Multimap expectedActions) { - - IJobUpdateDetails details = storage.read(new Work.Quiet() { + private IJobUpdateDetails getDetails() { + return storage.read(new Work.Quiet() { @Override public IJobUpdateDetails apply(StoreProvider storeProvider) { return storeProvider.getJobUpdateStore().fetchJobUpdateDetails(UPDATE_ID).get(); } }); + } + + private void assertLatestUpdateMessage(String expected) { + IJobUpdateDetails details = getDetails(); + assertEquals(expected, Iterables.getLast(details.getUpdateEvents()).getMessage()); + } + + private void assertState( + JobUpdateStatus expected, + Multimap expectedActions) { + + IJobUpdateDetails details = getDetails(); Iterable orderedEvents = EVENT_ORDER.sortedCopy(details.getInstanceEvents()); Multimap eventsByInstance = @@ -429,6 +439,7 @@ public class JobUpdaterIT extends EasyMockTest { clock.advance(Amount.of(PULSE_TIMEOUT_MS, Time.MILLISECONDS)); actions.put(2, INSTANCE_UPDATING); assertState(ROLL_FORWARD_AWAITING_PULSE, actions.build()); + assertLatestUpdateMessage(JobUpdateControllerImpl.PULSE_TIMEOUT_MESSAGE); // Pulse arrives and instance 2 is updated. assertEquals(JobUpdatePulseStatus.OK, updater.pulse(UPDATE_ID)); @@ -758,6 +769,7 @@ public class JobUpdaterIT extends EasyMockTest { // Instance 2 is rolled back. assertState(ROLLING_BACK, actions.build()); + assertLatestUpdateMessage(JobUpdateControllerImpl.failureMessage(2, Failure.EXITED)); changeState(JOB, 2, ASSIGNED, STARTING, RUNNING); actions.putAll(1, INSTANCE_ROLLING_BACK) .putAll(2, INSTANCE_ROLLED_BACK); @@ -928,6 +940,7 @@ public class JobUpdaterIT extends EasyMockTest { ImmutableMultimap.Builder actions = ImmutableMultimap.builder(); actions.putAll(0, INSTANCE_UPDATING); assertState(ERROR, actions.build()); + assertLatestUpdateMessage(JobUpdateControllerImpl.LOST_LOCK_MESSAGE); } private void expectInvalid(JobUpdate update) @@ -1141,6 +1154,7 @@ public class JobUpdaterIT extends EasyMockTest { clock.advance(RUNNING_TIMEOUT); actions.putAll(1, INSTANCE_UPDATE_FAILED, INSTANCE_ROLLING_BACK); assertState(ROLLING_BACK, actions.build()); + assertLatestUpdateMessage(JobUpdateControllerImpl.failureMessage(1, Failure.STUCK)); // Instance 1 is reverted. changeState(JOB, 1, ASSIGNED, STARTING, RUNNING); http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/48dc4c7a/src/test/java/org/apache/aurora/scheduler/updater/OneWayJobUpdaterTest.java ---------------------------------------------------------------------- diff --git a/src/test/java/org/apache/aurora/scheduler/updater/OneWayJobUpdaterTest.java b/src/test/java/org/apache/aurora/scheduler/updater/OneWayJobUpdaterTest.java index 7d0a743..d32b40a 100644 --- a/src/test/java/org/apache/aurora/scheduler/updater/OneWayJobUpdaterTest.java +++ b/src/test/java/org/apache/aurora/scheduler/updater/OneWayJobUpdaterTest.java @@ -21,6 +21,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.twitter.common.testing.easymock.EasyMockTest; +import org.apache.aurora.scheduler.updater.StateEvaluator.Failure; import org.apache.aurora.scheduler.updater.strategy.UpdateStrategy; import org.junit.Before; import org.junit.Test; @@ -35,7 +36,7 @@ import static org.apache.aurora.scheduler.updater.SideEffect.InstanceUpdateStatu import static org.apache.aurora.scheduler.updater.StateEvaluator.Result; import static org.apache.aurora.scheduler.updater.StateEvaluator.Result.EVALUATE_AFTER_MIN_RUNNING_MS; import static org.apache.aurora.scheduler.updater.StateEvaluator.Result.EVALUATE_ON_STATE_CHANGE; -import static org.apache.aurora.scheduler.updater.StateEvaluator.Result.FAILED; +import static org.apache.aurora.scheduler.updater.StateEvaluator.Result.FAILED_TERMINATED; import static org.apache.aurora.scheduler.updater.StateEvaluator.Result.KILL_TASK_AND_EVALUATE_ON_STATE_CHANGE; import static org.apache.aurora.scheduler.updater.StateEvaluator.Result.REPLACE_TASK_AND_EVALUATE_ON_STATE_CHANGE; import static org.apache.aurora.scheduler.updater.StateEvaluator.Result.SUCCEEDED; @@ -110,13 +111,15 @@ public class OneWayJobUpdaterTest extends EasyMockTest { private static SideEffect sideEffect(InstanceAction action, InstanceUpdateStatus... statuses) { return new SideEffect( Optional.of(action), - ImmutableSet.builder().add(statuses).build()); + ImmutableSet.builder().add(statuses).build(), + Optional.absent()); } private static SideEffect sideEffect(InstanceUpdateStatus... statuses) { return new SideEffect( Optional.absent(), - ImmutableSet.builder().add(statuses).build()); + ImmutableSet.builder().add(statuses).build(), + Optional.absent()); } @Test @@ -193,7 +196,7 @@ public class OneWayJobUpdaterTest extends EasyMockTest { 0, instance0, s0, - FAILED); + FAILED_TERMINATED); expectFetchAndEvaluate( 1, instance1,