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 F1BE717550 for ; Tue, 6 Jan 2015 19:55:20 +0000 (UTC) Received: (qmail 24798 invoked by uid 500); 6 Jan 2015 19:55:22 -0000 Delivered-To: apmail-aurora-commits-archive@aurora.apache.org Received: (qmail 24764 invoked by uid 500); 6 Jan 2015 19:55:21 -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 24755 invoked by uid 99); 6 Jan 2015 19:55:21 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 06 Jan 2015 19:55:21 +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, 06 Jan 2015 19:55:19 +0000 Received: (qmail 20766 invoked by uid 99); 6 Jan 2015 19:54:59 -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, 06 Jan 2015 19:54:59 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 7D9D59D1414; Tue, 6 Jan 2015 19:54:59 +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: <2529e7190d9548708ec3f286f73dd58d@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: incubator-aurora git commit: Remove SANDBOX_DELETED state. Date: Tue, 6 Jan 2015 19:54:59 +0000 (UTC) X-Virus-Checked: Checked by ClamAV on apache.org Repository: incubator-aurora Updated Branches: refs/heads/master aae3783f5 -> 1c53b9481 Remove SANDBOX_DELETED state. Bugs closed: AURORA-832 Reviewed at https://reviews.apache.org/r/28913/ Project: http://git-wip-us.apache.org/repos/asf/incubator-aurora/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-aurora/commit/1c53b948 Tree: http://git-wip-us.apache.org/repos/asf/incubator-aurora/tree/1c53b948 Diff: http://git-wip-us.apache.org/repos/asf/incubator-aurora/diff/1c53b948 Branch: refs/heads/master Commit: 1c53b94810e5b778ebf9c52da29f06b6a1cf7c29 Parents: aae3783 Author: Bill Farner Authored: Tue Jan 6 11:54:25 2015 -0800 Committer: Bill Farner Committed: Tue Jan 6 11:54:25 2015 -0800 ---------------------------------------------------------------------- .../thrift/org/apache/aurora/gen/api.thrift | 8 ++---- .../scheduler/storage/StorageBackfill.java | 15 ---------- .../apache/aurora/executor/gc_executor.py | 5 +--- .../apache/aurora/scheduler/base/JobsTest.java | 12 ++------ .../scheduler/state/TaskStateMachineTest.java | 12 ++------ .../scheduler/storage/StorageBackfillTest.java | 30 -------------------- 6 files changed, 8 insertions(+), 74 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/1c53b948/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 5665c69..08ba1cd 100644 --- a/api/src/main/thrift/org/apache/aurora/gen/api.thrift +++ b/api/src/main/thrift/org/apache/aurora/gen/api.thrift @@ -359,10 +359,7 @@ enum ScheduleStatus { /** A fault in the task environment has caused the system to believe the task no longer exists. * This can happen, for example, when a slave process disappears. */ - LOST = 7, - // TODO(maxim): Remove SANDBOX_DELETED in 0.7.0. (AURORA-832) - /** The task sandbox has been deleted by the executor. */ - SANDBOX_DELETED = 10 + LOST = 7 } // States that a task may be in while still considered active. @@ -396,8 +393,7 @@ const set LIVE_STATES = [ScheduleStatus.KILLING, const set TERMINAL_STATES = [ScheduleStatus.FAILED, ScheduleStatus.FINISHED, ScheduleStatus.KILLED, - ScheduleStatus.LOST, - ScheduleStatus.SANDBOX_DELETED] + ScheduleStatus.LOST] // Regular expressions for matching valid identifiers for job path components. All expressions // below should accept and reject the same set of inputs. http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/1c53b948/src/main/java/org/apache/aurora/scheduler/storage/StorageBackfill.java ---------------------------------------------------------------------- diff --git a/src/main/java/org/apache/aurora/scheduler/storage/StorageBackfill.java b/src/main/java/org/apache/aurora/scheduler/storage/StorageBackfill.java index 3e3543a..1814658 100644 --- a/src/main/java/org/apache/aurora/scheduler/storage/StorageBackfill.java +++ b/src/main/java/org/apache/aurora/scheduler/storage/StorageBackfill.java @@ -13,7 +13,6 @@ */ package org.apache.aurora.scheduler.storage; -import java.util.List; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; import java.util.logging.Logger; @@ -107,19 +106,6 @@ public final class StorageBackfill { } } - private static void rewriteSandboxDeletedState(ScheduledTask task) { - if (task.getStatus() == ScheduleStatus.SANDBOX_DELETED) { - List events = task.getTaskEvents(); - ScheduleStatus previousStatus = events.get(events.size() - 2).getStatus(); - - // Set the status to the previous event and drop the last event. - task.setStatus(previousStatus); - events.remove(events.size() - 1); - - LOG.info("Rewriting SANDBOX_DELETED status to " + previousStatus + " for " + Tasks.id(task)); - } - } - private static void populateJobKey(TaskConfig config, AtomicLong counter) { if (!config.isSetJob() || !JobKeys.isValid(IJobKey.build(config.getJob()))) { config.setJob(new JobKey() @@ -162,7 +148,6 @@ public final class StorageBackfill { // TODO(ksweeney): Guarantee tasks pass current validation code here and quarantine if they // don't. guaranteeShardUniqueness(builder, storeProvider.getUnsafeTaskStore(), clock); - rewriteSandboxDeletedState(builder); return IScheduledTask.build(builder); } }); http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/1c53b948/src/main/python/apache/aurora/executor/gc_executor.py ---------------------------------------------------------------------- diff --git a/src/main/python/apache/aurora/executor/gc_executor.py b/src/main/python/apache/aurora/executor/gc_executor.py index 9c24771..952d77d 100644 --- a/src/main/python/apache/aurora/executor/gc_executor.py +++ b/src/main/python/apache/aurora/executor/gc_executor.py @@ -278,12 +278,9 @@ class ThermosGCExecutor(ExecutorBase, ExceptionalThread, Observable): states = self.get_states(task_id) if states: _, last_state = states[-1] - # TODO(maxim): The SANDBOX_DELETED below is used as a safety fallback in case an unknown - # status is encountered during reconciliation. Unless GC executor is removed first - # (AURORA-715), consider a different fallback when removing the state updates[task_id] = THERMOS_TO_TWITTER_STATES.get( last_state, - ScheduleStatus.SANDBOX_DELETED) + ScheduleStatus.LOST) self.send_update( driver, task_id, http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/1c53b948/src/test/java/org/apache/aurora/scheduler/base/JobsTest.java ---------------------------------------------------------------------- diff --git a/src/test/java/org/apache/aurora/scheduler/base/JobsTest.java b/src/test/java/org/apache/aurora/scheduler/base/JobsTest.java index 56d49ea..a170fb6 100644 --- a/src/test/java/org/apache/aurora/scheduler/base/JobsTest.java +++ b/src/test/java/org/apache/aurora/scheduler/base/JobsTest.java @@ -13,14 +13,11 @@ */ package org.apache.aurora.scheduler.base; -import java.util.Arrays; -import java.util.Set; +import java.util.EnumSet; import com.google.common.base.Function; import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Sets; import org.apache.aurora.gen.JobStats; import org.apache.aurora.gen.ScheduleStatus; @@ -36,14 +33,9 @@ public class JobsTest { @Test public void testGetJobStats() { - // TODO(maxim): Drop when AURORA-832 is fixed. - Set statusValues = Sets.difference( - Sets.immutableEnumSet(Arrays.asList(ScheduleStatus.values())), - ImmutableSet.of(ScheduleStatus.SANDBOX_DELETED)); - ImmutableList tasks = FluentIterable - .from(statusValues) + .from(EnumSet.allOf(ScheduleStatus.class)) .transform(new Function() { @Override public IScheduledTask apply(ScheduleStatus status) { http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/1c53b948/src/test/java/org/apache/aurora/scheduler/state/TaskStateMachineTest.java ---------------------------------------------------------------------- diff --git a/src/test/java/org/apache/aurora/scheduler/state/TaskStateMachineTest.java b/src/test/java/org/apache/aurora/scheduler/state/TaskStateMachineTest.java index 50d387d..b30a073 100644 --- a/src/test/java/org/apache/aurora/scheduler/state/TaskStateMachineTest.java +++ b/src/test/java/org/apache/aurora/scheduler/state/TaskStateMachineTest.java @@ -88,10 +88,7 @@ public class TaskStateMachineTest { @Test public void testPostTerminalTransitionDenied() { - Set terminalStates = - Sets.difference(Tasks.TERMINAL_STATES, ImmutableSet.of(ScheduleStatus.SANDBOX_DELETED)); - - for (ScheduleStatus endState : terminalStates) { + for (ScheduleStatus endState : Tasks.TERMINAL_STATES) { stateMachine = makeStateMachine(makeTask(false)); Set finalActions = Sets.newHashSet(Action.SAVE_STATE); @@ -119,7 +116,7 @@ public class TaskStateMachineTest { expectUpdateStateOnTransitionTo(PENDING, ASSIGNED, STARTING, RUNNING); legalTransition(TaskState.valueOf(endState.name()), finalActions); - for (ScheduleStatus badTransition : terminalStates) { + for (ScheduleStatus badTransition : Tasks.TERMINAL_STATES) { illegalTransition(TaskState.valueOf(badTransition.name())); } } @@ -127,10 +124,7 @@ public class TaskStateMachineTest { @Test public void testTerminalToDeleted() { - Set terminalStates = - Sets.difference(Tasks.TERMINAL_STATES, ImmutableSet.of(ScheduleStatus.SANDBOX_DELETED)); - - for (ScheduleStatus endState : terminalStates) { + for (ScheduleStatus endState : Tasks.TERMINAL_STATES) { stateMachine = makeStateMachine(makeTask(false)); Set finalActions = Sets.newHashSet(Action.SAVE_STATE); http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/1c53b948/src/test/java/org/apache/aurora/scheduler/storage/StorageBackfillTest.java ---------------------------------------------------------------------- diff --git a/src/test/java/org/apache/aurora/scheduler/storage/StorageBackfillTest.java b/src/test/java/org/apache/aurora/scheduler/storage/StorageBackfillTest.java index 64c1b7f..7eafe07 100644 --- a/src/test/java/org/apache/aurora/scheduler/storage/StorageBackfillTest.java +++ b/src/test/java/org/apache/aurora/scheduler/storage/StorageBackfillTest.java @@ -43,16 +43,13 @@ import org.apache.aurora.scheduler.storage.entities.IJobConfiguration; import org.apache.aurora.scheduler.storage.entities.IJobKey; import org.apache.aurora.scheduler.storage.entities.IScheduledTask; import org.apache.aurora.scheduler.storage.entities.ITaskConfig; -import org.apache.aurora.scheduler.storage.entities.ITaskEvent; import org.apache.aurora.scheduler.storage.mem.MemStorage; import org.junit.Before; import org.junit.Test; -import static org.apache.aurora.gen.ScheduleStatus.FINISHED; import static org.apache.aurora.gen.ScheduleStatus.KILLED; import static org.apache.aurora.gen.ScheduleStatus.PENDING; import static org.apache.aurora.gen.ScheduleStatus.RUNNING; -import static org.apache.aurora.gen.ScheduleStatus.SANDBOX_DELETED; import static org.junit.Assert.assertEquals; public class StorageBackfillTest { @@ -77,33 +74,6 @@ public class StorageBackfillTest { } @Test - public void testRewriteSandboxDeleted() throws Exception { - final TaskConfig storedTask = defaultTask(); - final TaskEvent expectedEvent = new TaskEvent(100, FINISHED); - - storage.write(new Storage.MutateWork.NoResult.Quiet() { - @Override - protected void execute(Storage.MutableStoreProvider storeProvider) { - storeProvider.getUnsafeTaskStore().saveTasks(ImmutableSet.of( - IScheduledTask.build(new ScheduledTask() - .setStatus(SANDBOX_DELETED) - .setTaskEvents(ImmutableList.of(expectedEvent, new TaskEvent(200, SANDBOX_DELETED))) - .setAssignedTask(new AssignedTask() - .setTaskId(TASK_ID) - .setInstanceId(0) - .setTask(storedTask))))); - } - }); - - backfill(); - - assertEquals(FINISHED, getTask(TASK_ID).getStatus()); - assertEquals( - ImmutableList.of(ITaskEvent.build(expectedEvent)), - getTask(TASK_ID).getTaskEvents()); - } - - @Test public void testLoadTasksFromStorage() throws Exception { final TaskConfig storedTask = defaultTask();