You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by dm...@apache.org on 2017/10/24 22:37:47 UTC

aurora git commit: Do not reserve agents for updates when constraints change.

Repository: aurora
Updated Branches:
  refs/heads/master 71bb41c9a -> 38476abdf


Do not reserve agents for updates when constraints change.

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


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

Branch: refs/heads/master
Commit: 38476abdf1eac5f70ca9ab8a1a871dd34692d24a
Parents: 71bb41c
Author: David McLaughlin <da...@dmclaughlin.com>
Authored: Tue Oct 24 15:33:41 2017 -0700
Committer: David McLaughlin <da...@dmclaughlin.com>
Committed: Tue Oct 24 15:33:41 2017 -0700

----------------------------------------------------------------------
 .../scheduler/updater/InstanceUpdater.java      |  7 ++++++-
 .../scheduler/updater/InstanceUpdaterTest.java  | 20 ++++++++++++++++++++
 2 files changed, 26 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/38476abd/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 282ead4..0aa4f36 100644
--- a/src/main/java/org/apache/aurora/scheduler/updater/InstanceUpdater.java
+++ b/src/main/java/org/apache/aurora/scheduler/updater/InstanceUpdater.java
@@ -123,6 +123,10 @@ class InstanceUpdater implements StateEvaluator<Optional<IScheduledTask>> {
         .greaterThanOrEqualTo(bagFromResources(desired.getResources()));
   }
 
+  private boolean constraintsMatch(ITaskConfig desired, ITaskConfig existing) {
+    return desired.getConstraints().equals(existing.getConstraints());
+  }
+
   private StateEvaluator.Result handleActualAndDesiredPresent(IScheduledTask actualState) {
     Preconditions.checkState(desiredState.isPresent());
     Preconditions.checkArgument(!actualState.getTaskEvents().isEmpty());
@@ -151,7 +155,8 @@ class InstanceUpdater implements StateEvaluator<Optional<IScheduledTask>> {
       // This is not the configuration that we would like to run.
       if (isKillable(status)) {
         // Task is active, kill it.
-        if (resourceFits(desiredState.get(), actualState.getAssignedTask().getTask())) {
+        if (resourceFits(desiredState.get(), actualState.getAssignedTask().getTask())
+            && constraintsMatch(desiredState.get(), actualState.getAssignedTask().getTask())) {
           // If the desired task fits into the existing offer, we reserve the offer.
           return KILL_TASK_WITH_RESERVATION_AND_EVALUATE_ON_STATE_CHANGE;
         } else {

http://git-wip-us.apache.org/repos/asf/aurora/blob/38476abd/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 8832308..30f3e4f 100644
--- a/src/test/java/org/apache/aurora/scheduler/updater/InstanceUpdaterTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/updater/InstanceUpdaterTest.java
@@ -25,10 +25,13 @@ import org.apache.aurora.common.quantity.Amount;
 import org.apache.aurora.common.quantity.Time;
 import org.apache.aurora.common.util.testing.FakeClock;
 import org.apache.aurora.gen.AssignedTask;
+import org.apache.aurora.gen.Constraint;
 import org.apache.aurora.gen.ScheduleStatus;
 import org.apache.aurora.gen.ScheduledTask;
 import org.apache.aurora.gen.TaskConfig;
+import org.apache.aurora.gen.TaskConstraint;
 import org.apache.aurora.gen.TaskEvent;
+import org.apache.aurora.gen.ValueConstraint;
 import org.apache.aurora.scheduler.base.Tasks;
 import org.apache.aurora.scheduler.storage.entities.IScheduledTask;
 import org.apache.aurora.scheduler.storage.entities.ITaskConfig;
@@ -61,6 +64,9 @@ public class InstanceUpdaterTest {
           .setResources(ImmutableSet.of(numCpus(1.0))));
   private static final ITaskConfig NEW_EXTRA_RESOURCES = ITaskConfig.build(new TaskConfig()
       .setResources(ImmutableSet.of(numCpus(2.0))));
+  private static final ITaskConfig NEW_DIFFERENT_CONSTRAINTS = ITaskConfig.build(new TaskConfig()
+      .setConstraints(ImmutableSet.of(new Constraint("different",
+          TaskConstraint.value(new ValueConstraint(false, ImmutableSet.of("test")))))));
 
   private static final Amount<Long, Time> MIN_RUNNING_TIME = Amount.of(1L, Time.MINUTES);
   private static final Amount<Long, Time> A_LONG_TIME = Amount.of(1L, Time.DAYS);
@@ -150,6 +156,20 @@ public class InstanceUpdaterTest {
   }
 
   @Test
+  public void testUpdateWithConstraintChange() {
+    TestFixture f = new TestFixture(NEW_DIFFERENT_CONSTRAINTS, 1);
+    f.setActualState(OLD);
+    f.evaluate(KILL_TASK_AND_EVALUATE_ON_STATE_CHANGE, RUNNING);
+    f.evaluate(EVALUATE_ON_STATE_CHANGE, KILLING);
+    f.evaluate(REPLACE_TASK_AND_EVALUATE_ON_STATE_CHANGE, FINISHED);
+    f.setActualState(NEW_DIFFERENT_CONSTRAINTS);
+    f.evaluate(EVALUATE_ON_STATE_CHANGE, PENDING, ASSIGNED, STARTING);
+    f.evaluate(EVALUATE_AFTER_MIN_RUNNING_MS, RUNNING);
+    f.advanceTime(MIN_RUNNING_TIME);
+    f.evaluateCurrentState(SUCCEEDED);
+  }
+
+  @Test
   public void testUpdateRetryOnTaskExit() {
     TestFixture f = new TestFixture(NEW, 1);
     f.setActualState(OLD);