You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by zm...@apache.org on 2017/02/13 20:11:33 UTC

aurora git commit: Displaying update id after 'Killed for job update' message for the update that resulted in the task getting killed.

Repository: aurora
Updated Branches:
  refs/heads/master ad3377ab2 -> 40d91feb7


Displaying update id after 'Killed for job update' message for the update that
resulted in the task getting killed.

Testing Done:
Tests:
------
aurora job create devcluster/www-data/devel/hello_world my_jobs/new_hello_world_job.aurora
aurora update start devcluster/www-data/devel/hello_world my_jobs/new_hello_world_job_update.aurora

Completed Task status information:
-----------------------------------
3 minutes ago - KILLED : Instructed to kill task.
02/09 19:52:53 LOCAL \u2022 PENDING
02/09 19:52:53 LOCAL \u2022 ASSIGNED
02/09 19:52:54 LOCAL \u2022 STARTING \u2022 Initializing sandbox.
02/09 19:52:55 LOCAL \u2022 RUNNING \u2022 No health-check defined, task is assumed healthy.
02/09 19:53:08 LOCAL \u2022 KILLING \u2022 Killed for job update : 900256bb-9cad-41d6-b330-d74a751239bf
02/09 19:53:10 LOCAL \u2022 KILLED \u2022 Instructed to kill task.

Build tests:
-------------
./build-support/jenkins/build.sh
./src/test/sh/org/apache/aurora/e2e/test_end_to_end.sh

Bugs closed: AURORA-1806

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


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

Branch: refs/heads/master
Commit: 40d91feb71b85c1ad31233a45b6d70603de224fc
Parents: ad3377a
Author: Abhishek Jain <aj...@binghamton.edu>
Authored: Mon Feb 13 12:11:13 2017 -0800
Committer: Zameer Manji <zm...@apache.org>
Committed: Mon Feb 13 12:11:13 2017 -0800

----------------------------------------------------------------------
 .../scheduler/updater/InstanceActionHandler.java   | 15 ++++++++++-----
 .../scheduler/updater/JobUpdateControllerImpl.java |  3 ++-
 .../aurora/scheduler/updater/AddTaskTest.java      | 17 +++++++++++++----
 .../aurora/scheduler/updater/KillTaskTest.java     | 17 +++++++++++++----
 4 files changed, 38 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/40d91feb/src/main/java/org/apache/aurora/scheduler/updater/InstanceActionHandler.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/updater/InstanceActionHandler.java b/src/main/java/org/apache/aurora/scheduler/updater/InstanceActionHandler.java
index 11ffa66..f25dc0c 100644
--- a/src/main/java/org/apache/aurora/scheduler/updater/InstanceActionHandler.java
+++ b/src/main/java/org/apache/aurora/scheduler/updater/InstanceActionHandler.java
@@ -28,6 +28,7 @@ import org.apache.aurora.scheduler.state.StateManager;
 import org.apache.aurora.scheduler.storage.entities.IInstanceKey;
 import org.apache.aurora.scheduler.storage.entities.IInstanceTaskConfig;
 import org.apache.aurora.scheduler.storage.entities.IJobUpdateInstructions;
+import org.apache.aurora.scheduler.storage.entities.IJobUpdateKey;
 import org.apache.aurora.scheduler.storage.entities.IRange;
 import org.apache.aurora.scheduler.storage.entities.IScheduledTask;
 import org.apache.aurora.scheduler.storage.entities.ITaskConfig;
@@ -44,7 +45,8 @@ interface InstanceActionHandler {
       IJobUpdateInstructions instructions,
       MutableStoreProvider storeProvider,
       StateManager stateManager,
-      JobUpdateStatus status);
+      JobUpdateStatus status,
+      IJobUpdateKey key);
 
   Logger LOG = LoggerFactory.getLogger(InstanceActionHandler.class);
 
@@ -84,7 +86,8 @@ interface InstanceActionHandler {
         IJobUpdateInstructions instructions,
         MutableStoreProvider storeProvider,
         StateManager stateManager,
-        JobUpdateStatus status) {
+        JobUpdateStatus status,
+        IJobUpdateKey key) {
 
       Optional<IScheduledTask> task = getExistingTask(storeProvider, instance);
       if (task.isPresent()) {
@@ -114,7 +117,8 @@ interface InstanceActionHandler {
         IJobUpdateInstructions instructions,
         MutableStoreProvider storeProvider,
         StateManager stateManager,
-        JobUpdateStatus status) {
+        JobUpdateStatus status,
+        IJobUpdateKey key) {
 
       Optional<IScheduledTask> task = getExistingTask(storeProvider, instance);
       if (task.isPresent()) {
@@ -124,7 +128,7 @@ interface InstanceActionHandler {
             Tasks.id(task.get()),
             Optional.absent(),
             ScheduleStatus.KILLING,
-            Optional.of("Killed for job update."));
+            Optional.of("Killed for job update " + key.getId()));
       } else {
         // Due to async event processing it's possible to have a race between task event
         // and it's deletion from the store. This is a perfectly valid case.
@@ -142,7 +146,8 @@ interface InstanceActionHandler {
         IJobUpdateInstructions instructions,
         MutableStoreProvider storeProvider,
         StateManager stateManager,
-        JobUpdateStatus status) {
+        JobUpdateStatus status,
+        IJobUpdateKey key) {
 
       return Optional.of(Amount.of(
           (long) instructions.getSettings().getMinWaitInInstanceRunningMs(),

http://git-wip-us.apache.org/repos/asf/aurora/blob/40d91feb/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 80e97d7..729c123 100644
--- a/src/main/java/org/apache/aurora/scheduler/updater/JobUpdateControllerImpl.java
+++ b/src/main/java/org/apache/aurora/scheduler/updater/JobUpdateControllerImpl.java
@@ -663,7 +663,8 @@ class JobUpdateControllerImpl implements JobUpdateController {
                 instructions,
                 storeProvider,
                 stateManager,
-                updaterStatus);
+                updaterStatus,
+                key);
             if (reevaluateDelay.isPresent()) {
               executor.schedule(
                   getDeferredEvaluator(instance, key),

http://git-wip-us.apache.org/repos/asf/aurora/blob/40d91feb/src/test/java/org/apache/aurora/scheduler/updater/AddTaskTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/updater/AddTaskTest.java b/src/test/java/org/apache/aurora/scheduler/updater/AddTaskTest.java
index 56c94b5..b2c4c66 100644
--- a/src/test/java/org/apache/aurora/scheduler/updater/AddTaskTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/updater/AddTaskTest.java
@@ -19,6 +19,7 @@ import org.apache.aurora.common.testing.easymock.EasyMockTest;
 import org.apache.aurora.gen.InstanceKey;
 import org.apache.aurora.gen.InstanceTaskConfig;
 import org.apache.aurora.gen.JobUpdateInstructions;
+import org.apache.aurora.gen.JobUpdateKey;
 import org.apache.aurora.gen.JobUpdateSettings;
 import org.apache.aurora.gen.JobUpdateStatus;
 import org.apache.aurora.gen.Range;
@@ -28,7 +29,9 @@ import org.apache.aurora.scheduler.base.Query;
 import org.apache.aurora.scheduler.base.TaskTestUtil;
 import org.apache.aurora.scheduler.state.StateManager;
 import org.apache.aurora.scheduler.storage.entities.IInstanceKey;
+import org.apache.aurora.scheduler.storage.entities.IJobKey;
 import org.apache.aurora.scheduler.storage.entities.IJobUpdateInstructions;
+import org.apache.aurora.scheduler.storage.entities.IJobUpdateKey;
 import org.apache.aurora.scheduler.storage.testing.StorageTestUtil;
 import org.junit.Before;
 import org.junit.Test;
@@ -42,8 +45,11 @@ public class AddTaskTest extends EasyMockTest {
           .setSettings(
               new JobUpdateSettings()
                   .setMinWaitInInstanceRunningMs(1000)));
+  private static final IJobKey JOB = JobKeys.from("role", "env", "job");
   private static final IInstanceKey INSTANCE =
-      IInstanceKey.build(new InstanceKey(JobKeys.from("role", "env", "job").newBuilder(), 0));
+      IInstanceKey.build(new InstanceKey(JOB.newBuilder(), 0));
+  private static final IJobUpdateKey UPDATE_ID =
+          IJobUpdateKey.build(new JobUpdateKey(JOB.newBuilder(), "update_id"));
 
   private StorageTestUtil storageUtil;
   private StateManager stateManager;
@@ -73,7 +79,8 @@ public class AddTaskTest extends EasyMockTest {
         INSTRUCTIONS,
         storageUtil.mutableStoreProvider,
         stateManager,
-        JobUpdateStatus.ROLLING_FORWARD);
+        JobUpdateStatus.ROLLING_FORWARD,
+        UPDATE_ID);
   }
 
   @Test
@@ -89,7 +96,8 @@ public class AddTaskTest extends EasyMockTest {
         INSTRUCTIONS,
         storageUtil.mutableStoreProvider,
         stateManager,
-        JobUpdateStatus.ROLLING_FORWARD);
+        JobUpdateStatus.ROLLING_FORWARD,
+        UPDATE_ID);
   }
 
   @Test(expected = IllegalStateException.class)
@@ -103,6 +111,7 @@ public class AddTaskTest extends EasyMockTest {
         INSTRUCTIONS,
         storageUtil.mutableStoreProvider,
         stateManager,
-        JobUpdateStatus.ROLLING_BACK);
+        JobUpdateStatus.ROLLING_BACK,
+        UPDATE_ID);
   }
 }

http://git-wip-us.apache.org/repos/asf/aurora/blob/40d91feb/src/test/java/org/apache/aurora/scheduler/updater/KillTaskTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/updater/KillTaskTest.java b/src/test/java/org/apache/aurora/scheduler/updater/KillTaskTest.java
index e5935f6..833fd62 100644
--- a/src/test/java/org/apache/aurora/scheduler/updater/KillTaskTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/updater/KillTaskTest.java
@@ -18,6 +18,7 @@ import com.google.common.base.Optional;
 import org.apache.aurora.common.testing.easymock.EasyMockTest;
 import org.apache.aurora.gen.InstanceKey;
 import org.apache.aurora.gen.JobUpdateInstructions;
+import org.apache.aurora.gen.JobUpdateKey;
 import org.apache.aurora.gen.JobUpdateSettings;
 import org.apache.aurora.gen.JobUpdateStatus;
 import org.apache.aurora.gen.ScheduleStatus;
@@ -27,7 +28,9 @@ import org.apache.aurora.scheduler.base.TaskTestUtil;
 import org.apache.aurora.scheduler.state.StateChangeResult;
 import org.apache.aurora.scheduler.state.StateManager;
 import org.apache.aurora.scheduler.storage.entities.IInstanceKey;
+import org.apache.aurora.scheduler.storage.entities.IJobKey;
 import org.apache.aurora.scheduler.storage.entities.IJobUpdateInstructions;
+import org.apache.aurora.scheduler.storage.entities.IJobUpdateKey;
 import org.apache.aurora.scheduler.storage.testing.StorageTestUtil;
 import org.junit.Before;
 import org.junit.Test;
@@ -40,8 +43,11 @@ public class KillTaskTest extends EasyMockTest {
           .setSettings(
               new JobUpdateSettings()
                   .setMinWaitInInstanceRunningMs(1000)));
+  private static final IJobKey JOB = JobKeys.from("role", "env", "job");
   private static final IInstanceKey INSTANCE =
-      IInstanceKey.build(new InstanceKey(JobKeys.from("role", "env", "job").newBuilder(), 0));
+      IInstanceKey.build(new InstanceKey(JOB.newBuilder(), 0));
+  private static final IJobUpdateKey UPDATE_ID =
+          IJobUpdateKey.build(new JobUpdateKey(JOB.newBuilder(), "update_id"));
 
   private StorageTestUtil storageUtil;
   private StateManager stateManager;
@@ -67,7 +73,8 @@ public class KillTaskTest extends EasyMockTest {
         id,
         Optional.absent(),
         ScheduleStatus.KILLING,
-        Optional.of("Killed for job update."))).andReturn(StateChangeResult.SUCCESS);
+        Optional.of("Killed for job update " + UPDATE_ID.getId())))
+            .andReturn(StateChangeResult.SUCCESS);
 
     control.replay();
 
@@ -76,7 +83,8 @@ public class KillTaskTest extends EasyMockTest {
         INSTRUCTIONS,
         storageUtil.mutableStoreProvider,
         stateManager,
-        JobUpdateStatus.ROLLING_BACK);
+        JobUpdateStatus.ROLLING_BACK,
+        UPDATE_ID);
   }
 
   @Test
@@ -90,6 +98,7 @@ public class KillTaskTest extends EasyMockTest {
         INSTRUCTIONS,
         storageUtil.mutableStoreProvider,
         stateManager,
-        JobUpdateStatus.ROLLING_BACK);
+        JobUpdateStatus.ROLLING_BACK,
+        UPDATE_ID);
   }
 }