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/06 18:43:16 UTC

aurora git commit: Add message parameter to killTasks

Repository: aurora
Updated Branches:
  refs/heads/master 7de6d34be -> f88b7f3bf


Add message parameter to killTasks

RPC's such as pauseJobUpdate include a parameter for "a user-specified message
to include with the induced job update state change." This diff provides a
similar optional parameter for the killTasks RPC, which allows users to indicate
the reason why a task was killed, and later inspect that reason when consuming
task events.

Example usage from Aurora CLI:
`$ aurora job killall devcluster/www-data/prod/hello --message "Some message"`

In the task event, the supplied message (if provided) is appended to the
existing template "Killed by <user>", separated by a newline. For the above
example, this looks like: "Killed by aurora\nSome message".

Testing Done:
Added a unit test in the scheduler, and a test in the client.

Also manually tested using the Vagrant environment.

Bugs closed: AURORA-1846

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


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

Branch: refs/heads/master
Commit: f88b7f3bf5b7a7db6e422e38cbf22cf809f8ff87
Parents: 7de6d34
Author: Cody Gibb <co...@gmail.com>
Authored: Mon Feb 6 10:43:01 2017 -0800
Committer: Zameer Manji <zm...@apache.org>
Committed: Mon Feb 6 10:43:01 2017 -0800

----------------------------------------------------------------------
 RELEASE-NOTES.md                                |  7 +++
 .../thrift/org/apache/aurora/gen/api.thrift     |  2 +-
 .../aurora/scheduler/thrift/AuditMessages.java  |  6 ++-
 .../thrift/SchedulerThriftInterface.java        |  8 +++-
 .../thrift/aop/AnnotatedAuroraAdmin.java        |  3 +-
 .../python/apache/aurora/client/api/__init__.py |  4 +-
 .../python/apache/aurora/client/cli/jobs.py     | 10 ++--
 .../apache/aurora/client/hooks/hooked_api.py    |  9 ++--
 .../http/api/security/HttpSecurityIT.java       | 21 ++++----
 .../ShiroAuthorizingParamInterceptorTest.java   |  4 +-
 .../scheduler/thrift/AuditMessagesTest.java     | 26 +++++++++-
 .../thrift/SchedulerThriftInterfaceTest.java    | 27 +++++++++--
 src/test/python/apache/aurora/api_util.py       |  2 +-
 .../aurora/client/api/test_scheduler_client.py  | 10 ++--
 .../apache/aurora/client/cli/test_kill.py       | 50 ++++++++++++++------
 .../aurora/client/hooks/test_hooked_api.py      |  2 +-
 .../aurora/client/hooks/test_non_hooked_api.py  |  6 +--
 .../sh/org/apache/aurora/e2e/test_end_to_end.sh | 10 +++-
 18 files changed, 146 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/f88b7f3b/RELEASE-NOTES.md
----------------------------------------------------------------------
diff --git a/RELEASE-NOTES.md b/RELEASE-NOTES.md
index 05fdfb6..3e98802 100644
--- a/RELEASE-NOTES.md
+++ b/RELEASE-NOTES.md
@@ -1,3 +1,10 @@
+0.18.0
+=====
+
+### New/updated:
+
+- Add message parameter to `killTasks` RPC.
+
 0.17.0
 ======
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/f88b7f3b/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 c3ec965..6205c2e 100644
--- a/api/src/main/thrift/org/apache/aurora/gen/api.thrift
+++ b/api/src/main/thrift/org/apache/aurora/gen/api.thrift
@@ -1075,7 +1075,7 @@ service AuroraSchedulerManager extends ReadOnlyScheduler {
   Response restartShards(5: JobKey job, 3: set<i32> shardIds)
 
   /** Initiates a kill on tasks. */
-  Response killTasks(4: JobKey job, 5: set<i32> instances)
+  Response killTasks(4: JobKey job, 5: set<i32> instances, 6: string message)
 
   /**
    * Adds new instances with the TaskConfig of the existing instance pointed by the key.

http://git-wip-us.apache.org/repos/asf/aurora/blob/f88b7f3b/src/main/java/org/apache/aurora/scheduler/thrift/AuditMessages.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/thrift/AuditMessages.java b/src/main/java/org/apache/aurora/scheduler/thrift/AuditMessages.java
index a7aeced..d7bf846 100644
--- a/src/main/java/org/apache/aurora/scheduler/thrift/AuditMessages.java
+++ b/src/main/java/org/apache/aurora/scheduler/thrift/AuditMessages.java
@@ -50,8 +50,10 @@ class AuditMessages {
     return com.google.common.base.Optional.of("Transition forced by " + getRemoteUserName());
   }
 
-  com.google.common.base.Optional<String> killedByRemoteUser() {
-    return com.google.common.base.Optional.of("Killed by " + getRemoteUserName());
+  com.google.common.base.Optional<String> killedByRemoteUser(
+      com.google.common.base.Optional<String> message) {
+    String suffix = message.transform(s -> ".\n" + s).or("");
+    return com.google.common.base.Optional.of("Killed by " + getRemoteUserName() + suffix);
   }
 
   com.google.common.base.Optional<String> restartedByRemoteUser() {

http://git-wip-us.apache.org/repos/asf/aurora/blob/f88b7f3b/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 a40114d..a211483 100644
--- a/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java
+++ b/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java
@@ -487,7 +487,11 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
   }
 
   @Override
-  public Response killTasks(@Nullable JobKey mutableJob, @Nullable Set<Integer> instances) {
+  public Response killTasks(
+      @Nullable JobKey mutableJob,
+      @Nullable Set<Integer> instances,
+      @Nullable String message) {
+
     Response response = empty();
     IJobKey jobKey = JobKeys.assertValid(IJobKey.build(mutableJob));
     Query.Builder query;
@@ -514,7 +518,7 @@ class SchedulerThriftInterface implements AnnotatedAuroraAdmin {
             taskId,
             Optional.absent(),
             ScheduleStatus.KILLING,
-            auditMessages.killedByRemoteUser())) {
+            auditMessages.killedByRemoteUser(Optional.fromNullable(message)))) {
           ++tasksKilled;
         }
       }

http://git-wip-us.apache.org/repos/asf/aurora/blob/f88b7f3b/src/main/java/org/apache/aurora/scheduler/thrift/aop/AnnotatedAuroraAdmin.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/thrift/aop/AnnotatedAuroraAdmin.java b/src/main/java/org/apache/aurora/scheduler/thrift/aop/AnnotatedAuroraAdmin.java
index bfc3dc8..d63b11d 100644
--- a/src/main/java/org/apache/aurora/scheduler/thrift/aop/AnnotatedAuroraAdmin.java
+++ b/src/main/java/org/apache/aurora/scheduler/thrift/aop/AnnotatedAuroraAdmin.java
@@ -61,7 +61,8 @@ public interface AnnotatedAuroraAdmin extends AuroraAdmin.Iface {
   @Override
   Response killTasks(
       @AuthorizingParam @Nullable JobKey job,
-      @Nullable Set<Integer> instances) throws TException;
+      @Nullable Set<Integer> instances,
+      @Nullable String message) throws TException;
 
   @Override
   Response addInstances(

http://git-wip-us.apache.org/repos/asf/aurora/blob/f88b7f3b/src/main/python/apache/aurora/client/api/__init__.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/api/__init__.py b/src/main/python/apache/aurora/client/api/__init__.py
index e1dde63..1250ccd 100644
--- a/src/main/python/apache/aurora/client/api/__init__.py
+++ b/src/main/python/apache/aurora/client/api/__init__.py
@@ -106,7 +106,7 @@ class AuroraClientAPI(object):
              % (count, job_key, instance_id))
     return self._scheduler_proxy.addInstances(key, count)
 
-  def kill_job(self, job_key, instances=None):
+  def kill_job(self, job_key, instances=None, message=None):
     log.info("Killing tasks for job: %s" % job_key)
     self._assert_valid_job_key(job_key)
 
@@ -114,7 +114,7 @@ class AuroraClientAPI(object):
       log.info("Instances to be killed: %s" % instances)
       instances = frozenset([int(s) for s in instances])
 
-    return self._scheduler_proxy.killTasks(job_key.to_thrift(), instances)
+    return self._scheduler_proxy.killTasks(job_key.to_thrift(), instances, message)
 
   def check_status(self, job_key):
     self._assert_valid_job_key(job_key)

http://git-wip-us.apache.org/repos/asf/aurora/blob/f88b7f3b/src/main/python/apache/aurora/client/cli/jobs.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/jobs.py b/src/main/python/apache/aurora/client/cli/jobs.py
index 8ba41aa..b79ae56 100644
--- a/src/main/python/apache/aurora/client/cli/jobs.py
+++ b/src/main/python/apache/aurora/client/cli/jobs.py
@@ -305,7 +305,9 @@ class AbstractKillCommand(Verb):
         CONFIG_OPTION,
         BATCH_OPTION,
         MAX_TOTAL_FAILURES_OPTION,
-        NO_BATCHING_OPTION]
+        NO_BATCHING_OPTION,
+        CommandOption('--message', '-m', type=str, default=None,
+                      help='Message to include with the kill state transition')]
 
   def wait_kill_tasks(self, context, scheduler, job_key, instances=None):
     monitor = JobMonitor(scheduler, job_key)
@@ -331,7 +333,7 @@ class AbstractKillCommand(Verb):
       batch = []
       for i in range(min(context.options.batch_size, len(instances_to_kill))):
         batch.append(instances_to_kill.pop())
-      resp = api.kill_job(job, batch, config=config)
+      resp = api.kill_job(job, batch, config=config, message=context.options.message)
       # Short circuit max errors in this case as it's most likely a fatal repeatable error.
       context.log_response_and_raise(
         resp,
@@ -412,7 +414,7 @@ class KillCommand(AbstractKillCommand):
     api = context.get_api(job.cluster)
     config = context.get_job_config_optional(job, context.options.config)
     if context.options.no_batching:
-      resp = api.kill_job(job, instances_arg, config=config)
+      resp = api.kill_job(job, instances_arg, config=config, message=context.options.message)
       context.log_response_and_raise(resp)
       wait_result = self.wait_kill_tasks(context, api.scheduler_proxy, job, instances_arg)
       if wait_result is not EXIT_OK:
@@ -442,7 +444,7 @@ class KillAllJobCommand(AbstractKillCommand):
     api = context.get_api(job.cluster)
     config = context.get_job_config_optional(job, context.options.config)
     if context.options.no_batching:
-      resp = api.kill_job(job, None, config=config)
+      resp = api.kill_job(job, None, config=config, message=context.options.message)
       context.log_response_and_raise(resp)
       wait_result = self.wait_kill_tasks(context, api.scheduler_proxy, job)
       if wait_result is not EXIT_OK:

http://git-wip-us.apache.org/repos/asf/aurora/blob/f88b7f3b/src/main/python/apache/aurora/client/hooks/hooked_api.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/hooks/hooked_api.py b/src/main/python/apache/aurora/client/hooks/hooked_api.py
index 86c3a09..63c8733 100644
--- a/src/main/python/apache/aurora/client/hooks/hooked_api.py
+++ b/src/main/python/apache/aurora/client/hooks/hooked_api.py
@@ -52,8 +52,9 @@ class NonHookedAuroraClientAPI(AuroraClientAPI):
   def add_instances(self, job_key, instance_id, count, config=None):
     return super(NonHookedAuroraClientAPI, self).add_instances(job_key, instance_id, count)
 
-  def kill_job(self, job_key, instances=None, config=None):
-    return super(NonHookedAuroraClientAPI, self).kill_job(job_key, instances=instances)
+  def kill_job(self, job_key, instances=None, config=None, message=None):
+    return super(NonHookedAuroraClientAPI, self).kill_job(job_key, instances=instances,
+                                                          message=message)
 
   def restart(self, job_key, shards, restart_settings, config=None):
     return super(NonHookedAuroraClientAPI, self).restart(job_key, shards, restart_settings)
@@ -162,10 +163,10 @@ class HookedAuroraClientAPI(NonHookedAuroraClientAPI):
         _partial(super(HookedAuroraClientAPI, self).add_instances,
             job_key, instance_id, count, config=config))
 
-  def kill_job(self, job_key, instances=None, config=None):
+  def kill_job(self, job_key, instances=None, config=None, message=None):
     return self._hooked_call(config, job_key,
         _partial(super(HookedAuroraClientAPI, self).kill_job,
-            job_key, instances=instances, config=config))
+            job_key, instances=instances, config=config, message=message))
 
   def restart(self, job_key, shards, restart_settings, config=None):
     return self._hooked_call(config, job_key,

http://git-wip-us.apache.org/repos/asf/aurora/blob/f88b7f3b/src/test/java/org/apache/aurora/scheduler/http/api/security/HttpSecurityIT.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/http/api/security/HttpSecurityIT.java b/src/test/java/org/apache/aurora/scheduler/http/api/security/HttpSecurityIT.java
index b20900d..d05eceb 100644
--- a/src/test/java/org/apache/aurora/scheduler/http/api/security/HttpSecurityIT.java
+++ b/src/test/java/org/apache/aurora/scheduler/http/api/security/HttpSecurityIT.java
@@ -225,7 +225,7 @@ public class HttpSecurityIT extends AbstractJettyTest {
 
   private void assertKillTasksFails(AuroraAdmin.Client client) throws TException {
     try {
-      client.killTasks(null, null);
+      client.killTasks(null, null, null);
       fail("killTasks should fail.");
     } catch (TTransportException e) {
       // Expected.
@@ -236,44 +236,45 @@ public class HttpSecurityIT extends AbstractJettyTest {
   public void testAuroraSchedulerManager() throws TException, ServletException, IOException {
     JobKey job = JobKeys.from("role", "env", "name").newBuilder();
 
-    expect(auroraAdmin.killTasks(job, null)).andReturn(OK).times(2);
-    expect(auroraAdmin.killTasks(ADS_STAGING_JOB.newBuilder(), null)).andReturn(OK);
+    expect(auroraAdmin.killTasks(job, null, null)).andReturn(OK).times(2);
+    expect(auroraAdmin.killTasks(ADS_STAGING_JOB.newBuilder(), null, null)).andReturn(OK);
     expectShiroAfterAuthFilter().atLeastOnce();
 
     replayAndStart();
 
     assertEquals(
         OK,
-        getAuthenticatedClient(WFARNER).killTasks(job, null));
+        getAuthenticatedClient(WFARNER).killTasks(job, null, null));
     assertEquals(
         OK,
-        getAuthenticatedClient(ROOT).killTasks(job, null));
+        getAuthenticatedClient(ROOT).killTasks(job, null, null));
 
     assertEquals(
         ResponseCode.INVALID_REQUEST,
-        getAuthenticatedClient(UNPRIVILEGED).killTasks(null, null).getResponseCode());
+        getAuthenticatedClient(UNPRIVILEGED).killTasks(null, null, null).getResponseCode());
     assertEquals(
         ResponseCode.AUTH_FAILED,
         getAuthenticatedClient(UNPRIVILEGED)
-            .killTasks(job, null)
+            .killTasks(job, null, null)
             .getResponseCode());
     assertEquals(
         ResponseCode.INVALID_REQUEST,
-        getAuthenticatedClient(BACKUP_SERVICE).killTasks(null, null).getResponseCode());
+        getAuthenticatedClient(BACKUP_SERVICE).killTasks(null, null, null).getResponseCode());
     assertEquals(
         ResponseCode.AUTH_FAILED,
         getAuthenticatedClient(BACKUP_SERVICE)
-            .killTasks(job, null)
+            .killTasks(job, null, null)
             .getResponseCode());
     assertEquals(
         ResponseCode.AUTH_FAILED,
         getAuthenticatedClient(DEPLOY_SERVICE)
-            .killTasks(job, null)
+            .killTasks(job, null, null)
             .getResponseCode());
     assertEquals(
         OK,
         getAuthenticatedClient(DEPLOY_SERVICE).killTasks(
             ADS_STAGING_JOB.newBuilder(),
+            null,
             null));
 
     assertKillTasksFails(getUnauthenticatedClient());

http://git-wip-us.apache.org/repos/asf/aurora/blob/f88b7f3b/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingParamInterceptorTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingParamInterceptorTest.java b/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingParamInterceptorTest.java
index 05f4a18..cda6ca2 100644
--- a/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingParamInterceptorTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingParamInterceptorTest.java
@@ -124,7 +124,7 @@ public class ShiroAuthorizingParamInterceptorTest extends EasyMockTest {
 
     assertEquals(
         ResponseCode.AUTH_FAILED,
-        decoratedThrift.killTasks(JOB_KEY.newBuilder(), null).getResponseCode());
+        decoratedThrift.killTasks(JOB_KEY.newBuilder(), null, null).getResponseCode());
   }
 
   @Test
@@ -135,7 +135,7 @@ public class ShiroAuthorizingParamInterceptorTest extends EasyMockTest {
         ResponseCode.INVALID_REQUEST,
         decoratedThrift.killTasks(
             JOB_KEY.newBuilder().setName(null),
-            null).getResponseCode());
+            null, null).getResponseCode());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/aurora/blob/f88b7f3b/src/test/java/org/apache/aurora/scheduler/thrift/AuditMessagesTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/thrift/AuditMessagesTest.java b/src/test/java/org/apache/aurora/scheduler/thrift/AuditMessagesTest.java
index 9c8460c..d9e935c 100644
--- a/src/test/java/org/apache/aurora/scheduler/thrift/AuditMessagesTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/thrift/AuditMessagesTest.java
@@ -22,6 +22,7 @@ import org.junit.Test;
 import static org.apache.aurora.scheduler.thrift.AuditMessages.DEFAULT_USER;
 import static org.easymock.EasyMock.expect;
 import static org.hamcrest.CoreMatchers.containsString;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 
 public class AuditMessagesTest extends EasyMockTest {
@@ -31,7 +32,9 @@ public class AuditMessagesTest extends EasyMockTest {
 
     control.replay();
 
-    assertThat(emptyMessages.killedByRemoteUser().get(), containsString(DEFAULT_USER));
+    assertThat(emptyMessages.killedByRemoteUser(
+        com.google.common.base.Optional.absent()).get(),
+        containsString(DEFAULT_USER));
     assertThat(emptyMessages.restartedByRemoteUser().get(), containsString(DEFAULT_USER));
     assertThat(emptyMessages.transitionedBy().get(), containsString(DEFAULT_USER));
   }
@@ -45,8 +48,27 @@ public class AuditMessagesTest extends EasyMockTest {
 
     control.replay();
 
-    assertThat(presentMessages.killedByRemoteUser().get(), containsString("shiro"));
+    assertThat(presentMessages.killedByRemoteUser(
+        com.google.common.base.Optional.absent()).get(),
+        containsString("shiro"));
     assertThat(presentMessages.restartedByRemoteUser().get(), containsString("shiro"));
     assertThat(presentMessages.transitionedBy().get(), containsString("shiro"));
   }
+
+  @Test
+  public void testKilledByRemoteUserMessages() {
+    Subject subject = createMock(Subject.class);
+    AuditMessages messages = new AuditMessages(() -> Optional.of(subject));
+
+    expect(subject.getPrincipal()).andReturn("shiro").times(2);
+
+    control.replay();
+
+    assertEquals(messages.killedByRemoteUser(
+        com.google.common.base.Optional.of("Test message")).get(),
+        "Killed by shiro.\nTest message");
+    assertEquals(messages.killedByRemoteUser(
+        com.google.common.base.Optional.absent()).get(),
+        "Killed by shiro");
+  }
 }

http://git-wip-us.apache.org/repos/asf/aurora/blob/f88b7f3b/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 b7574e4..0cdd982 100644
--- a/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
@@ -697,7 +697,11 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
   }
 
   private void expectTransitionsToKilling() {
-    expect(auditMessages.killedByRemoteUser()).andReturn(Optional.of("test"));
+    expectTransitionsToKilling(Optional.absent());
+  }
+
+  private void expectTransitionsToKilling(Optional<String> message) {
+    expect(auditMessages.killedByRemoteUser(message)).andReturn(Optional.of("test"));
     expect(stateManager.changeState(
         storageUtil.mutableStoreProvider,
         TASK_ID,
@@ -715,7 +719,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertOkResponse(thrift.killTasks(JOB_KEY.newBuilder(), null));
+    assertOkResponse(thrift.killTasks(JOB_KEY.newBuilder(), null, null));
     assertEquals(1L, statsProvider.getLongValue(KILL_TASKS));
   }
 
@@ -728,7 +732,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    assertOkResponse(thrift.killTasks(JOB_KEY.newBuilder(), ImmutableSet.of(1)));
+    assertOkResponse(thrift.killTasks(JOB_KEY.newBuilder(), ImmutableSet.of(1), null));
     assertEquals(1L, statsProvider.getLongValue(KILL_TASKS));
   }
 
@@ -747,7 +751,7 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     assertResponse(
         LOCK_ERROR,
-        thrift.killTasks(JOB_KEY.newBuilder(), null));
+        thrift.killTasks(JOB_KEY.newBuilder(), null, null));
     assertEquals(0L, statsProvider.getLongValue(KILL_TASKS));
   }
 
@@ -758,13 +762,26 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
     control.replay();
 
-    Response response = thrift.killTasks(JOB_KEY.newBuilder(), null);
+    Response response = thrift.killTasks(JOB_KEY.newBuilder(), null, null);
     assertOkResponse(response);
     assertMessageMatches(response, SchedulerThriftInterface.NO_TASKS_TO_KILL_MESSAGE);
     assertEquals(0L, statsProvider.getLongValue(KILL_TASKS));
   }
 
   @Test
+  public void testKillTasksWithMessage() throws Exception {
+    String message = "Test message";
+    Query.Builder query = Query.unscoped().byJob(JOB_KEY).active();
+    storageUtil.expectTaskFetch(query, buildScheduledTask());
+    lockManager.assertNotLocked(LOCK_KEY);
+    expectTransitionsToKilling(Optional.of(message));
+
+    control.replay();
+
+    assertOkResponse(thrift.killTasks(JOB_KEY.newBuilder(), null, message));
+  }
+
+  @Test
   public void testSetQuota() throws Exception {
     ResourceAggregate resourceAggregate = new ResourceAggregate()
         .setNumCpus(10)

http://git-wip-us.apache.org/repos/asf/aurora/blob/f88b7f3b/src/test/python/apache/aurora/api_util.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/api_util.py b/src/test/python/apache/aurora/api_util.py
index 983cde4..f4935b5 100644
--- a/src/test/python/apache/aurora/api_util.py
+++ b/src/test/python/apache/aurora/api_util.py
@@ -85,7 +85,7 @@ class SchedulerThriftApiSpec(ReadOnlyScheduler.Iface):
   def restartShards(self, job, shardIds):
     pass
 
-  def killTasks(self, jobKey, instances):
+  def killTasks(self, jobKey, instances, message):
     pass
 
   def addInstances(self, key, count):

http://git-wip-us.apache.org/repos/asf/aurora/blob/f88b7f3b/src/test/python/apache/aurora/client/api/test_scheduler_client.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/api/test_scheduler_client.py b/src/test/python/apache/aurora/client/api/test_scheduler_client.py
index f6018ca..fab9798 100644
--- a/src/test/python/apache/aurora/client/api/test_scheduler_client.py
+++ b/src/test/python/apache/aurora/client/api/test_scheduler_client.py
@@ -138,9 +138,9 @@ class TestSchedulerProxyInjection(unittest.TestCase):
     self.make_scheduler_proxy().getJobs(ROLE)
 
   def test_killTasks(self):
-    self.mock_thrift_client.killTasks(IsA(JobKey), IgnoreArg()).AndReturn(DEFAULT_RESPONSE)
+    self.mock_thrift_client.killTasks(IsA(JobKey), IgnoreArg(), None).AndReturn(DEFAULT_RESPONSE)
     self.mox.ReplayAll()
-    self.make_scheduler_proxy().killTasks(JobKey(), {0})
+    self.make_scheduler_proxy().killTasks(JobKey(), {0}, None)
 
   def test_getQuota(self):
     self.mock_thrift_client.getQuota(IgnoreArg()).AndReturn(DEFAULT_RESPONSE)
@@ -195,11 +195,11 @@ class TestSchedulerProxyInjection(unittest.TestCase):
     self.make_scheduler_proxy().pulseJobUpdate('update_id')
 
   def test_raise_auth_error(self):
-    self.mock_thrift_client.killTasks(None, None).AndRaise(TRequestsTransport.AuthError())
+    self.mock_thrift_client.killTasks(None, None, None).AndRaise(TRequestsTransport.AuthError())
     self.mock_scheduler_client.get_failed_auth_message().AndReturn('failed auth')
     self.mox.ReplayAll()
     with pytest.raises(scheduler_client.SchedulerProxy.AuthError):
-      self.make_scheduler_proxy().killTasks(None, None)
+      self.make_scheduler_proxy().killTasks(None, None, None)
 
 
 class TestSchedulerProxyAdminInjection(TestSchedulerProxyInjection):
@@ -471,7 +471,7 @@ class TestSchedulerClient(unittest.TestCase):
     client.get.return_value = mock_scheduler_client
 
     proxy = scheduler_client.SchedulerProxy(Cluster(name='local'))
-    proxy.killTasks(JobKey(), None)
+    proxy.killTasks(JobKey(), None, None)
 
     assert mock_thrift_client.killTasks.call_count == 3
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/f88b7f3b/src/test/python/apache/aurora/client/cli/test_kill.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/test_kill.py b/src/test/python/apache/aurora/client/cli/test_kill.py
index 564ea44..269b566 100644
--- a/src/test/python/apache/aurora/client/cli/test_kill.py
+++ b/src/test/python/apache/aurora/client/cli/test_kill.py
@@ -70,7 +70,8 @@ class TestKillCommand(AuroraClientCommandTest):
     mock_api.kill_job.assert_called_once_with(
         self.TEST_JOBKEY,
         mock_options.instance_spec.instance,
-        config=None)
+        config=None,
+        message=None)
 
     self.assert_lock_message(fake_context)
 
@@ -98,7 +99,8 @@ class TestKillCommand(AuroraClientCommandTest):
     mock_api.kill_job.assert_called_once_with(
         self.TEST_JOBKEY,
         mock_options.instance_spec.instance,
-        config=None)
+        config=None,
+        message=None)
 
     self.assert_lock_message(fake_context)
 
@@ -174,7 +176,7 @@ class TestKillCommand(AuroraClientCommandTest):
     command.execute(fake_context)
 
     assert mock_api.kill_job.mock_calls == [
-        call(self.TEST_JOBKEY, mock_options.instance_spec.instance, config=config)
+        call(self.TEST_JOBKEY, mock_options.instance_spec.instance, config=config, message=None)
     ]
 
   def test_kill_batched_passes_config(self):
@@ -197,11 +199,10 @@ class TestKillCommand(AuroraClientCommandTest):
 
     mock_api = fake_context.get_api('test')
     mock_api.kill_job.return_value = self.create_simple_success_response()
-
     command.execute(fake_context)
 
     assert mock_api.kill_job.mock_calls == [
-        call(self.TEST_JOBKEY, mock_options.instance_spec.instance, config=config)
+        call(self.TEST_JOBKEY, mock_options.instance_spec.instance, config=config, message=None)
     ]
 
 
@@ -226,7 +227,9 @@ class TestKillAllCommand(AuroraClientCommandTest):
 
     command.execute(fake_context)
 
-    assert mock_api.kill_job.mock_calls == [call(self.TEST_JOBKEY, None, config=config)]
+    assert mock_api.kill_job.mock_calls == [
+        call(self.TEST_JOBKEY, None, config=config, message=None)
+    ]
 
 
 class TestClientKillCommand(AuroraClientCommandTest):
@@ -246,11 +249,14 @@ class TestClientKillCommand(AuroraClientCommandTest):
     return mock_monitor
 
   @classmethod
-  def assert_kill_calls(cls, api, instance_range=None, instances=None):
+  def assert_kill_calls(cls, api, instance_range=None, instances=None, message=None):
     if instances:
-      kill_calls = [call(cls.TEST_JOBKEY, instances, config=None)]
+      kill_calls = [call(cls.TEST_JOBKEY, instances, config=None, message=message)]
     else:
-      kill_calls = [call(cls.TEST_JOBKEY, [i], config=None) for i in instance_range]
+      kill_calls = [
+          call(cls.TEST_JOBKEY, [i], config=None, message=message)
+          for i in instance_range
+      ]
     assert api.kill_job.mock_calls == kill_calls
 
   @classmethod
@@ -263,7 +269,7 @@ class TestClientKillCommand(AuroraClientCommandTest):
 
   @classmethod
   def assert_kill_call_no_instances(cls, api):
-    assert api.kill_job.mock_calls == [call(cls.TEST_JOBKEY, None, config=None)]
+    assert api.kill_job.mock_calls == [call(cls.TEST_JOBKEY, None, config=None, message=None)]
 
   @classmethod
   def assert_query(cls, fake_api):
@@ -303,7 +309,7 @@ class TestClientKillCommand(AuroraClientCommandTest):
       cmd = AuroraCommandLine()
       cmd.execute(['job', 'killall', self.TEST_JOBSPEC])
 
-      self.assert_kill_calls(api, instance_range=range(20))
+      self.assert_kill_calls(api, instance_range=range(20), message=None)
       self.assert_wait_calls(mock_monitor, m.terminal, instance_range=range(20))
       self.assert_query(api)
 
@@ -320,7 +326,7 @@ class TestClientKillCommand(AuroraClientCommandTest):
       cmd.execute(['job', 'kill', '--no-batching', self.get_instance_spec('0,2,4-6')])
 
       instances = [0, 2, 4, 5, 6]
-      self.assert_kill_calls(api, instances=instances)
+      self.assert_kill_calls(api, instances=instances, message=None)
       self.assert_wait_calls(mock_monitor, m.terminal, instances=instances)
 
   def test_kill_job_with_invalid_instances_strict(self):
@@ -348,7 +354,7 @@ class TestClientKillCommand(AuroraClientCommandTest):
       cmd = AuroraCommandLine()
       cmd.execute(['job', 'kill', '--no-batching', self.get_instance_spec('0,2,4-6,11-13')])
       instances = [0, 2, 4, 5, 6, 11, 12, 13]
-      self.assert_kill_calls(api, instances=instances)
+      self.assert_kill_calls(api, instances=instances, message=None)
       self.assert_wait_calls(mock_monitor, m.terminal, instances=instances)
 
   def test_kill_job_with_instances_batched(self):
@@ -386,7 +392,7 @@ class TestClientKillCommand(AuroraClientCommandTest):
       cmd.execute(['job', 'kill', '--max-total-failures=1', self.get_instance_spec('0-4')])
 
       # We should have aborted after the second batch.
-      self.assert_kill_calls(api, instance_range=range(2))
+      self.assert_kill_calls(api, instance_range=range(2), message=None)
       self.assert_query(api)
 
   def test_kill_job_with_empty_instances_batched(self):
@@ -455,3 +461,19 @@ class TestClientKillCommand(AuroraClientCommandTest):
          'Instances [0, 2, 4, 5, 6] were not killed in time',
          'Instances [7, 8, 9, 10, 11] were not killed in time',
          'Exceeded maximum number of errors while killing instances']
+
+  def test_kill_job_with_message(self):
+    """Test kill client-side API logic."""
+    mock_context = FakeAuroraCommandContext()
+    mock_monitor = self.get_monitor_mock()
+    with contextlib.nested(
+        patch('apache.aurora.client.cli.jobs.Job.create_context', return_value=mock_context),
+        patch('apache.aurora.client.cli.jobs.JobMonitor', return_value=mock_monitor)) as (_, m):
+      api = mock_context.get_api('west')
+      api.kill_job.return_value = self.create_simple_success_response()
+      cmd = AuroraCommandLine()
+      message = 'Test message'
+      cmd.execute(['job', 'kill', '--no-batch', '--message', message, self.get_instance_spec('0')])
+
+      instances = [0]
+      self.assert_kill_calls(api, instances=instances, message=message)

http://git-wip-us.apache.org/repos/asf/aurora/blob/f88b7f3b/src/test/python/apache/aurora/client/hooks/test_hooked_api.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/hooks/test_hooked_api.py b/src/test/python/apache/aurora/client/hooks/test_hooked_api.py
index eb97c61..10027b0 100644
--- a/src/test/python/apache/aurora/client/hooks/test_hooked_api.py
+++ b/src/test/python/apache/aurora/client/hooks/test_hooked_api.py
@@ -58,7 +58,7 @@ def test_api_methods_params(method_name):
     assert api_argspec.varargs == nonhooked_argspec.varargs
     assert api_argspec.keywords == nonhooked_argspec.keywords
     assert len(api_argspec.args) + 1 == len(nonhooked_argspec.args)
-    assert nonhooked_argspec.args[len(api_argspec.args)] == 'config'
+    assert 'config' in nonhooked_argspec.args
     if api_argspec.defaults is None:
       assert len(nonhooked_argspec.defaults) == 1
       assert nonhooked_argspec.defaults[0] is None

http://git-wip-us.apache.org/repos/asf/aurora/blob/f88b7f3b/src/test/python/apache/aurora/client/hooks/test_non_hooked_api.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/hooks/test_non_hooked_api.py b/src/test/python/apache/aurora/client/hooks/test_non_hooked_api.py
index 04b2257..59eca76 100644
--- a/src/test/python/apache/aurora/client/hooks/test_non_hooked_api.py
+++ b/src/test/python/apache/aurora/client/hooks/test_non_hooked_api.py
@@ -41,8 +41,8 @@ class TestNonHookedAuroraClientAPI(unittest.TestCase):
 
     class FakeAuroraClientAPI(object):
 
-      def kill_job(self, job_key, instances=None):
-        test_obj.API_CALL = functools.partial(self.kill_job, job_key, instances)
+      def kill_job(self, job_key, instances=None, message=None):
+        test_obj.API_CALL = functools.partial(self.kill_job, job_key, instances, message)
         return test_obj.RETURN_VALUE
 
       def restart(self, job_key, shards, restart_settings):
@@ -82,7 +82,7 @@ class TestNonHookedAuroraClientAPI(unittest.TestCase):
       self.test_job_key,
       self.test_shards,
       config=self.test_config)
-    self._verify_api_call(return_value, self.test_job_key, self.test_shards)
+    self._verify_api_call(return_value, self.test_job_key, self.test_shards, None)
 
   def test_restart_discards_config(self):
     return_value = self.api.restart(

http://git-wip-us.apache.org/repos/asf/aurora/blob/f88b7f3b/src/test/sh/org/apache/aurora/e2e/test_end_to_end.sh
----------------------------------------------------------------------
diff --git a/src/test/sh/org/apache/aurora/e2e/test_end_to_end.sh b/src/test/sh/org/apache/aurora/e2e/test_end_to_end.sh
index 736d1fc..80b4c54 100755
--- a/src/test/sh/org/apache/aurora/e2e/test_end_to_end.sh
+++ b/src/test/sh/org/apache/aurora/e2e/test_end_to_end.sh
@@ -269,9 +269,11 @@ test_run() {
 
 test_kill() {
   local _jobkey=$1
+  shift 1
+  local _extra_args="${@}"
 
-  aurora job kill $_jobkey/1
-  aurora job killall $_jobkey
+  aurora job kill $_jobkey/1 $_extra_args
+  aurora job killall $_jobkey $_extra_args
 }
 
 test_quota() {
@@ -563,6 +565,8 @@ TEST_DAEMONIZING_PROCESS_ARGS=(
   $TEST_DAEMONIZING_PROCESS_CONFIG_FILE
 )
 
+TEST_JOB_KILL_MESSAGE_ARGS=("${TEST_JOB_ARGS[@]}" "--message='Test message'")
+
 trap collect_result EXIT
 
 aurorabuild all
@@ -578,6 +582,8 @@ test_http_example_basic "${TEST_JOB_REVOCABLE_ARGS[@]}"
 
 test_http_example_basic "${TEST_JOB_GPU_ARGS[@]}"
 
+test_http_example_basic "${TEST_JOB_KILL_MESSAGE_ARGS[@]}"
+
 test_http_example "${TEST_JOB_DOCKER_ARGS[@]}"
 
 setup_image_stores