You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by ma...@apache.org on 2014/05/08 02:51:01 UTC

git commit: Moving kill wait to the client (Part 2: server changes)

Repository: incubator-aurora
Updated Branches:
  refs/heads/master a2becf17f -> e9ca57937


Moving kill wait to the client (Part 2: server changes)

Bugs closed: AURORA-370

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


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

Branch: refs/heads/master
Commit: e9ca579378fc613662dc9eae9ab4dfa0e99b530c
Parents: a2becf1
Author: Maxim Khutornenko <ma...@apache.org>
Authored: Wed May 7 17:50:45 2014 -0700
Committer: Maxim Khutornenko <ma...@apache.org>
Committed: Wed May 7 17:50:45 2014 -0700

----------------------------------------------------------------------
 .../thrift/SchedulerThriftInterface.java        | 55 ++------------------
 .../thrift/SchedulerThriftInterfaceTest.java    | 28 ++--------
 2 files changed, 7 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/e9ca5793/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 abf47d7..9bb5c25 100644
--- a/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java
+++ b/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java
@@ -44,13 +44,7 @@ import com.google.common.collect.Maps;
 import com.google.common.collect.Multimap;
 import com.google.common.collect.Multimaps;
 import com.google.common.collect.Sets;
-import com.twitter.common.args.Arg;
-import com.twitter.common.args.CmdLine;
 import com.twitter.common.base.MorePreconditions;
-import com.twitter.common.base.Supplier;
-import com.twitter.common.quantity.Amount;
-import com.twitter.common.quantity.Time;
-import com.twitter.common.util.BackoffHelper;
 
 import org.apache.aurora.auth.CapabilityValidator;
 import org.apache.aurora.auth.CapabilityValidator.AuditCheck;
@@ -150,16 +144,6 @@ import static org.apache.aurora.gen.apiConstants.CURRENT_API_VERSION;
 class SchedulerThriftInterface implements AuroraAdmin.Iface {
   private static final Logger LOG = Logger.getLogger(SchedulerThriftInterface.class.getName());
 
-  @CmdLine(name = "kill_task_initial_backoff",
-      help = "Initial backoff delay while waiting for the tasks to transition to KILLED.")
-  private static final Arg<Amount<Long, Time>> KILL_TASK_INITIAL_BACKOFF =
-      Arg.create(Amount.of(1L, Time.SECONDS));
-
-  @CmdLine(name = "kill_task_max_backoff",
-      help = "Max backoff delay while waiting for the tasks to transition to KILLED.")
-  private static final Arg<Amount<Long, Time>> KILL_TASK_MAX_BACKOFF =
-      Arg.create(Amount.of(1L, Time.MINUTES));
-
   private static final Function<IScheduledTask, String> GET_ROLE = Functions.compose(
       new Function<ITaskConfig, String>() {
         @Override
@@ -179,8 +163,6 @@ class SchedulerThriftInterface implements AuroraAdmin.Iface {
   private final CronJobManager cronJobManager;
   private final CronPredictor cronPredictor;
   private final QuotaManager quotaManager;
-  private final Amount<Long, Time> killTaskInitialBackoff;
-  private final Amount<Long, Time> killTaskMaxBackoff;
 
   @Inject
   SchedulerThriftInterface(
@@ -204,9 +186,7 @@ class SchedulerThriftInterface implements AuroraAdmin.Iface {
         maintenance,
         cronJobManager,
         cronPredictor,
-        quotaManager,
-        KILL_TASK_INITIAL_BACKOFF.get(),
-        KILL_TASK_MAX_BACKOFF.get());
+        quotaManager);
   }
 
   @VisibleForTesting
@@ -220,9 +200,7 @@ class SchedulerThriftInterface implements AuroraAdmin.Iface {
       MaintenanceController maintenance,
       CronJobManager cronJobManager,
       CronPredictor cronPredictor,
-      QuotaManager quotaManager,
-      Amount<Long, Time> initialBackoff,
-      Amount<Long, Time> maxBackoff) {
+      QuotaManager quotaManager) {
 
     this.storage = checkNotNull(storage);
     this.schedulerCore = checkNotNull(schedulerCore);
@@ -234,8 +212,6 @@ class SchedulerThriftInterface implements AuroraAdmin.Iface {
     this.cronJobManager = checkNotNull(cronJobManager);
     this.cronPredictor = checkNotNull(cronPredictor);
     this.quotaManager = checkNotNull(quotaManager);
-    this.killTaskInitialBackoff = checkNotNull(initialBackoff);
-    this.killTaskMaxBackoff = checkNotNull(maxBackoff);
   }
 
   @Override
@@ -568,32 +544,7 @@ class SchedulerThriftInterface implements AuroraAdmin.Iface {
       return response.setResponseCode(LOCK_ERROR).setMessage(e.getMessage());
     }
 
-    // TODO(William Farner): Move this into the client.
-    BackoffHelper backoff = new BackoffHelper(killTaskInitialBackoff, killTaskMaxBackoff, true);
-    final Query.Builder activeQuery = Query.arbitrary(query.setStatuses(Tasks.ACTIVE_STATES));
-    try {
-      backoff.doUntilSuccess(new Supplier<Boolean>() {
-        @Override
-        public Boolean get() {
-          Set<IScheduledTask> tasks = Storage.Util.consistentFetchTasks(storage, activeQuery);
-          if (tasks.isEmpty()) {
-            LOG.info("Tasks all killed, done waiting.");
-            return true;
-          } else {
-            LOG.info("Jobs not yet killed, waiting...");
-            return false;
-          }
-        }
-      });
-      response.setResponseCode(OK).setMessage("Tasks killed.");
-    } catch (InterruptedException e) {
-      LOG.warning("Interrupted while trying to kill tasks: " + e);
-      Thread.currentThread().interrupt();
-      response.setResponseCode(ERROR).setMessage("killTasks thread was interrupted.");
-    } catch (BackoffHelper.BackoffStoppedException e) {
-      response.setResponseCode(ERROR).setMessage("Tasks were not killed in time.");
-    }
-    return response;
+    return response.setResponseCode(OK);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/e9ca5793/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 169c403..47d2fd6 100644
--- a/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
@@ -339,9 +339,8 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     Query.Builder query = Query.unscoped().byJob(JOB_KEY).active();
     expectAuth(ROOT, false);
     expectAuth(ROLE, true);
-    storageUtil.expectTaskFetch(query, buildScheduledTask(JOB_NAME)).times(2);
+    storageUtil.expectTaskFetch(query, buildScheduledTask(JOB_NAME));
     scheduler.killTasks(query, USER);
-    storageUtil.expectTaskFetch(query);
     lockManager.validateIfLocked(LOCK_KEY, Optional.of(LOCK));
 
     control.replay();
@@ -350,22 +349,6 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
   }
 
   @Test
-  public void testKillTasksDelayed() throws Exception {
-    Query.Builder query = Query.unscoped().byJob(JOB_KEY).active();
-    IScheduledTask task = buildScheduledTask(JOB_NAME);
-    expectAuth(ROOT, false);
-    expectAuth(ROLE, true);
-    scheduler.killTasks(query, USER);
-    storageUtil.expectTaskFetch(query, task).times(2);
-    storageUtil.expectTaskFetch(query);
-    lockManager.validateIfLocked(LOCK_KEY, Optional.<ILock>absent());
-
-    control.replay();
-
-    assertOkResponse(thrift.killTasks(query.get(), DEFAULT_LOCK, SESSION));
-  }
-
-  @Test
   public void testKillTasksLockCheckFailed() throws Exception {
     Query.Builder query = Query.unscoped().byJob(JOB_KEY).active();
     IScheduledTask task2 = buildScheduledTask("job_bar");
@@ -401,7 +384,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     expectAuth(ROOT, true);
     scheduler.killTasks(query, USER);
-    storageUtil.expectTaskFetch(query).times(2);
+    storageUtil.expectTaskFetch(query);
 
     control.replay();
 
@@ -426,7 +409,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     expectAuth(ROOT, true);
 
     scheduler.killTasks(query, USER);
-    storageUtil.expectTaskFetch(query).times(2);
+    storageUtil.expectTaskFetch(query);
 
     control.replay();
 
@@ -509,9 +492,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
   @Test
   public void testMachineMaintenanceAccessDenied() throws Exception {
-    Hosts hosts = new Hosts()
-        .setHostNames(ImmutableSet.of("host1"));
-    Set<HostStatus> statuses = ImmutableSet.of();
+    Hosts hosts = new Hosts().setHostNames(ImmutableSet.of("host1"));
 
     expectAuth(ROOT, false).times(4);
     expectAuth(MACHINE_MAINTAINER, false).times(4);
@@ -1269,7 +1250,6 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     storageUtil.expectTaskFetch(query, buildScheduledTask(JOB_NAME));
     scheduler.killTasks(query, USER);
-    storageUtil.expectTaskFetch(query.active());
     lockManager.validateIfLocked(LOCK_KEY, Optional.<ILock>absent());
 
     control.replay();