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 2015/05/08 23:47:15 UTC

aurora git commit: Adding missing API hook support in kill commands.

Repository: aurora
Updated Branches:
  refs/heads/master c34e50c79 -> a2e952b92


Adding missing API hook support in kill commands.

Bugs closed: AURORA-1307

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


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

Branch: refs/heads/master
Commit: a2e952b92e1e104034f3983d998ad52211b3211f
Parents: c34e50c
Author: Maxim Khutornenko <ma...@apache.org>
Authored: Fri May 8 14:41:38 2015 -0700
Committer: Maxim Khutornenko <ma...@apache.org>
Committed: Fri May 8 14:41:38 2015 -0700

----------------------------------------------------------------------
 .../python/apache/aurora/client/cli/context.py  |   4 +
 .../python/apache/aurora/client/cli/jobs.py     |  31 ++-
 .../python/apache/aurora/client/cli/options.py  |   8 +-
 .../aurora/client/cli/test_cancel_update.py     |  32 +--
 .../apache/aurora/client/cli/test_kill.py       | 222 ++++++++++---------
 .../python/apache/aurora/client/cli/util.py     |  10 +
 6 files changed, 163 insertions(+), 144 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/a2e952b9/src/main/python/apache/aurora/client/cli/context.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/context.py b/src/main/python/apache/aurora/client/cli/context.py
index bd40410..c850223 100644
--- a/src/main/python/apache/aurora/client/cli/context.py
+++ b/src/main/python/apache/aurora/client/cli/context.py
@@ -100,6 +100,10 @@ class AuroraCommandContext(Context):
       apis[cluster] = api
     return add_auth_error_handler(apis[cluster])
 
+  def get_job_config_optional(self, jobkey, config_file):
+    """Loads a job configuration if provided."""
+    return self.get_job_config(jobkey, config_file) if config_file is not None else None
+
   def get_job_config(self, jobkey, config_file):
     """Loads a job configuration from a config file."""
     jobname = jobkey.name

http://git-wip-us.apache.org/repos/asf/aurora/blob/a2e952b9/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 6d82a7e..5f60841 100644
--- a/src/main/python/apache/aurora/client/cli/jobs.py
+++ b/src/main/python/apache/aurora/client/cli/jobs.py
@@ -295,9 +295,9 @@ class InspectCommand(Verb):
 class AbstractKillCommand(Verb):
   def get_options(self):
     return [BROWSER_OPTION,
-        CommandOption("--config", type=str, default=None, dest="config",
-            metavar="pathname",
-            help="Config file for the job, possibly containing hooks"),
+        BIND_OPTION,
+        JSON_READ_OPTION,
+        CONFIG_OPTION,
         BATCH_OPTION,
         MAX_TOTAL_FAILURES_OPTION,
         NO_BATCHING_OPTION]
@@ -309,7 +309,7 @@ class AbstractKillCommand(Verb):
       return EXIT_TIMEOUT
     return EXIT_OK
 
-  def kill_in_batches(self, context, job, instances_arg):
+  def kill_in_batches(self, context, job, instances_arg, config):
     api = context.get_api(job.cluster)
     # query the job, to get the list of active instances.
     tasks = context.get_active_instances(job)
@@ -326,7 +326,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)
+      resp = api.kill_job(job, batch, config=config)
       # Short circuit max errors in this case as it's most likely a fatal repeatable error.
       context.log_response_and_raise(
         resp,
@@ -365,14 +365,15 @@ class KillCommand(AbstractKillCommand):
     if context.options.strict:
       context.verify_instances_option_validity(job, instances_arg)
     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)
+      resp = api.kill_job(job, instances_arg, config=config)
       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:
         return wait_result
     else:
-      self.kill_in_batches(context, job, instances_arg)
+      self.kill_in_batches(context, job, instances_arg, config)
     if context.options.open_browser:
       webbrowser.open_new_tab(get_job_page(api, job))
     context.print_out("Job kill succeeded")
@@ -394,14 +395,15 @@ class KillAllJobCommand(AbstractKillCommand):
   def execute(self, context):
     job = context.options.jobspec
     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)
+      resp = api.kill_job(job, None, config=config)
       context.log_response_and_raise(resp)
       wait_result = self.wait_kill_tasks(context, api.scheduler_proxy, job)
       if wait_result is not EXIT_OK:
         return wait_result
     else:
-      self.kill_in_batches(context, job, None)
+      self.kill_in_batches(context, job, None, config)
     if context.options.open_browser:
       webbrowser.open_new_tab(get_job_page(api, job))
     context.print_out("Job killall succeeded")
@@ -500,8 +502,7 @@ class RestartCommand(Verb):
     if instances is not None and context.options.strict:
       context.verify_instances_option_validity(job, instances)
     api = context.get_api(job.cluster)
-    config = (context.get_job_config(job, context.options.config)
-        if context.options.config else None)
+    config = context.get_job_config_optional(job, context.options.config)
     updater_config = UpdaterConfig(
         context.options.batch_size,
         context.options.restart_threshold,
@@ -687,16 +688,14 @@ class CancelUpdateCommand(Verb):
 
   def get_options(self):
     return [JSON_READ_OPTION,
-        CommandOption("--config", type=str, default=None, dest="config_file",
-            metavar="pathname",
-            help="Config file for the job, possibly containing hooks"),
+        BIND_OPTION,
+        CONFIG_OPTION,
         JOBSPEC_ARGUMENT]
 
   def execute(self, context):
     context.print_err(CLIENT_UPDATER_DEPRECATION)
     api = context.get_api(context.options.jobspec.cluster)
-    config = (context.get_job_config(context.options.jobspec, context.options.config_file)
-        if context.options.config_file else None)
+    config = context.get_job_config_optional(context.options.jobspec, context.options.config)
     resp = api.cancel_update(context.options.jobspec, config=config)
     context.log_response_and_raise(resp)
     return EXIT_OK

http://git-wip-us.apache.org/repos/asf/aurora/blob/a2e952b9/src/main/python/apache/aurora/client/cli/options.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/options.py b/src/main/python/apache/aurora/client/cli/options.py
index d175e8c..07344b3 100644
--- a/src/main/python/apache/aurora/client/cli/options.py
+++ b/src/main/python/apache/aurora/client/cli/options.py
@@ -168,14 +168,16 @@ BIND_OPTION = CommandOption('--bind', dest='bindings',
 
 BROWSER_OPTION = CommandOption('--open-browser', default=False, dest='open_browser',
     action='store_true',
-    help='open browser to view job page after job is created')
+    help='Open browser to view job page after job is created')
 
 
 CONFIG_ARGUMENT = CommandOption('config_file', type=str, metavar="pathname",
-    help='pathname of the aurora configuration file contain the job specification')
+    help='Pathname of the aurora configuration file contain the job specification')
+
 
 CONFIG_OPTION = CommandOption('--config', type=str, default=None, metavar="pathname",
-    help='pathname of the aurora configuration file contain the job specification')
+    help='Pathname of the aurora configuration file containing job specification'
+        'and possibly API hook definitions')
 
 EXECUTOR_SANDBOX_OPTION = CommandOption('--executor-sandbox', action='store_true',
      default=False, help='Run the command in the executor sandbox instead of the task sandbox')

http://git-wip-us.apache.org/repos/asf/aurora/blob/a2e952b9/src/test/python/apache/aurora/client/cli/test_cancel_update.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/test_cancel_update.py b/src/test/python/apache/aurora/client/cli/test_cancel_update.py
index f9facb6..d4fc049 100644
--- a/src/test/python/apache/aurora/client/cli/test_cancel_update.py
+++ b/src/test/python/apache/aurora/client/cli/test_cancel_update.py
@@ -12,28 +12,16 @@
 # limitations under the License.
 #
 
-from mock import patch
-from twitter.common.contextutil import temporary_file
+from mock import call, patch
 
 from apache.aurora.client.cli.client import AuroraCommandLine
-from apache.aurora.common.aurora_job_key import AuroraJobKey
 
 from .util import AuroraClientCommandTest, FakeAuroraCommandContext
 
-from gen.apache.aurora.api.ttypes import JobKey, TaskQuery
+from gen.apache.aurora.api.ttypes import JobKey, Lock, LockKey, LockValidation, TaskQuery
 
 
 class TestClientCancelUpdateCommand(AuroraClientCommandTest):
-
-  @classmethod
-  def assert_cancel_update_called(cls, mock_api):
-    # Running cancel update should result in calling the API cancel_update
-    # method once, with an AuroraJobKey parameter.
-    assert mock_api.cancel_update.call_count == 1
-    mock_api.cancel_update.assert_called_with(
-        AuroraJobKey(cls.TEST_CLUSTER, cls.TEST_ROLE, cls.TEST_ENV, cls.TEST_JOB),
-        config=None)
-
   def test_simple_successful_cancel_update(self):
     """Run a test of the "kill" command against a mocked-out API:
     Verifies that the kill command sends the right API RPCs, and performs the correct
@@ -43,8 +31,8 @@ class TestClientCancelUpdateCommand(AuroraClientCommandTest):
     mock_api.cancel_update.return_value = self.create_simple_success_response()
     with patch('apache.aurora.client.cli.jobs.Job.create_context', return_value=mock_context):
       cmd = AuroraCommandLine()
-      cmd.execute(['job', 'cancel-update', 'west/bozo/test/hello'])
-      self.assert_cancel_update_called(mock_api)
+      cmd.execute(['job', 'cancel-update', self.TEST_JOBSPEC])
+      assert mock_api.cancel_update.mock_calls == [call(self.TEST_JOBKEY, config=None)]
 
   @classmethod
   def get_expected_task_query(cls, shards=None):
@@ -65,14 +53,10 @@ class TestClientCancelUpdateCommand(AuroraClientCommandTest):
     (mock_api, mock_scheduler_proxy) = self.create_mock_api()
     mock_scheduler_proxy.releaseLock.return_value = self.get_release_lock_response()
     with patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy):
-      with temporary_file() as fp:
-        fp.write(self.get_valid_config())
-        fp.flush()
-        cmd = AuroraCommandLine()
-        cmd.execute(['job', 'cancel-update', 'west/mchucarroll/test/hello'])
+      cmd = AuroraCommandLine()
+      cmd.execute(['job', 'cancel-update', self.TEST_JOBSPEC])
 
       # All that cancel_update really does is release the update lock.
       # So that's all we really need to check.
-      assert mock_scheduler_proxy.releaseLock.call_count == 1
-      assert mock_scheduler_proxy.releaseLock.call_args[0][0].key.job == JobKey(environment='test',
-          role='mchucarroll', name='hello')
+      assert mock_scheduler_proxy.releaseLock.mock_calls == [
+          call(Lock(key=LockKey(job=self.TEST_JOBKEY.to_thrift())), LockValidation.UNCHECKED)]

http://git-wip-us.apache.org/repos/asf/aurora/blob/a2e952b9/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 d6dde5f..5eae70a 100644
--- a/src/test/python/apache/aurora/client/cli/test_kill.py
+++ b/src/test/python/apache/aurora/client/cli/test_kill.py
@@ -17,14 +17,12 @@ import unittest
 
 import pytest
 from mock import call, Mock, patch
-from twitter.common.contextutil import temporary_file
 
 from apache.aurora.client.api.job_monitor import JobMonitor
 from apache.aurora.client.cli import Context
 from apache.aurora.client.cli.client import AuroraCommandLine
-from apache.aurora.client.cli.jobs import KillCommand
+from apache.aurora.client.cli.jobs import KillAllJobCommand, KillCommand
 from apache.aurora.client.cli.options import parse_instances, TaskInstanceKey
-from apache.aurora.common.aurora_job_key import AuroraJobKey
 
 from .util import AuroraClientCommandTest, FakeAuroraCommandContext, mock_verb_options
 
@@ -55,11 +53,8 @@ class TestKillCommand(AuroraClientCommandTest):
   def test_kill_lock_error_nobatch(self):
     """Verify that the no batch code path correctly includes the lock error message."""
     command = KillCommand()
-
-    jobkey = AuroraJobKey("cluster", "role", "env", "job")
-
     mock_options = mock_verb_options(command)
-    mock_options.instance_spec = TaskInstanceKey(jobkey, [])
+    mock_options.instance_spec = TaskInstanceKey(self.TEST_JOBKEY, [])
     mock_options.no_batching = True
 
     fake_context = FakeAuroraCommandContext()
@@ -72,17 +67,18 @@ class TestKillCommand(AuroraClientCommandTest):
     with pytest.raises(Context.CommandError):
       command.execute(fake_context)
 
-    mock_api.kill_job.assert_called_once_with(jobkey, mock_options.instance_spec.instance)
+    mock_api.kill_job.assert_called_once_with(
+        self.TEST_JOBKEY,
+        mock_options.instance_spec.instance,
+        config=None)
+
     self.assert_lock_message(fake_context)
 
   def test_kill_lock_error_batches(self):
     """Verify that the batch kill path short circuits and includes the lock error message."""
     command = KillCommand()
-
-    jobkey = AuroraJobKey("cluster", "role", "env", "job")
-
     mock_options = mock_verb_options(command)
-    mock_options.instance_spec = TaskInstanceKey(jobkey, [1])
+    mock_options.instance_spec = TaskInstanceKey(self.TEST_JOBKEY, [1])
     mock_options.no_batching = False
 
     fake_context = FakeAuroraCommandContext()
@@ -99,17 +95,18 @@ class TestKillCommand(AuroraClientCommandTest):
     with pytest.raises(Context.CommandError):
       command.execute(fake_context)
 
-    mock_api.kill_job.assert_called_once_with(jobkey, mock_options.instance_spec.instance)
+    mock_api.kill_job.assert_called_once_with(
+        self.TEST_JOBKEY,
+        mock_options.instance_spec.instance,
+        config=None)
+
     self.assert_lock_message(fake_context)
 
   def test_kill_inactive_instance_spec(self):
     """Verify the instance spec is validated in a batched kill."""
     command = KillCommand()
-
-    jobkey = AuroraJobKey("cluster", "role", "env", "job")
-
     mock_options = mock_verb_options(command)
-    mock_options.instance_spec = TaskInstanceKey(jobkey, [1])
+    mock_options.instance_spec = TaskInstanceKey(self.TEST_JOBKEY, [1])
     mock_options.no_batching = False
     mock_options.strict = True
 
@@ -125,11 +122,8 @@ class TestKillCommand(AuroraClientCommandTest):
   def test_kill_batched_queries_active_instances(self):
     """Verify that the batch kill operates on active instances only."""
     command = KillCommand()
-
-    jobkey = AuroraJobKey("cluster", "role", "env", "job")
-
     mock_options = mock_verb_options(command)
-    mock_options.instance_spec = TaskInstanceKey(jobkey, [1])
+    mock_options.instance_spec = TaskInstanceKey(self.TEST_JOBKEY, [1])
     mock_options.no_batching = False
 
     fake_context = FakeAuroraCommandContext()
@@ -138,16 +132,14 @@ class TestKillCommand(AuroraClientCommandTest):
     fake_context.add_expected_query_result(AuroraClientCommandTest.create_empty_task_result())
 
     command.execute(fake_context)
-    assert fake_context.get_err()[0] == "No tasks to kill found for job cluster/role/env/job"
+    message = "No tasks to kill found for job %s" % self.TEST_JOBKEY.to_path()
+    assert fake_context.get_err()[0] == message
 
   def test_kill_opens_url(self):
     """Verify the kill commands opens the job page if requested"""
     command = KillCommand()
-
-    jobkey = AuroraJobKey("cluster", "role", "env", "job")
-
     mock_options = mock_verb_options(command)
-    mock_options.instance_spec = TaskInstanceKey(jobkey, [1])
+    mock_options.instance_spec = TaskInstanceKey(self.TEST_JOBKEY, [1])
     mock_options.open_browser = True
 
     fake_context = FakeAuroraCommandContext()
@@ -158,9 +150,84 @@ class TestKillCommand(AuroraClientCommandTest):
     command.execute(fake_context)
 
     assert self.mock_webbrowser.mock_calls == [
-        call("http://something_or_other/scheduler/role/env/job")
+        call("http://something_or_other/scheduler/bozo/test/hello")
     ]
 
+  def test_kill_nobatch_passes_config(self):
+    """Verify that the no batch code path correctly passes job config to the api."""
+    command = KillCommand()
+    config = self.get_valid_config()
+
+    mock_options = mock_verb_options(command)
+    mock_options.instance_spec = TaskInstanceKey(self.TEST_JOBKEY, [])
+    mock_options.no_batching = True
+    mock_options.config = config
+
+    fake_context = FakeAuroraCommandContext()
+    fake_context.set_options(mock_options)
+    fake_context.add_config(config)
+
+    mock_api = fake_context.get_api('test')
+    fake_context.add_expected_query_result(AuroraClientCommandTest.create_empty_task_result())
+    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)
+    ]
+
+  def test_kill_batched_passes_config(self):
+    """Verify that the batched code path correctly passes job config to the api."""
+    command = KillCommand()
+    config = self.get_valid_config()
+
+    mock_options = mock_verb_options(command)
+    mock_options.instance_spec = TaskInstanceKey(self.TEST_JOBKEY, [1])
+    mock_options.no_batching = False
+    mock_options.config = config
+
+    fake_context = FakeAuroraCommandContext()
+    fake_context.set_options(mock_options)
+    fake_context.add_config(config)
+
+    fake_context.add_expected_query_result(
+      AuroraClientCommandTest.create_query_call_result(
+        AuroraClientCommandTest.create_scheduled_task(1, ScheduleStatus.RUNNING)))
+
+    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)
+    ]
+
+
+class TestKillAllCommand(AuroraClientCommandTest):
+  def test_killall_nobatch_passes_config(self):
+    """Verify that the no batch code path correctly passes job config to the api."""
+    command = KillAllJobCommand()
+    config = self.get_valid_config()
+
+    mock_options = mock_verb_options(command)
+    mock_options.jobspec = self.TEST_JOBKEY
+    mock_options.no_batching = True
+    mock_options.config = config
+
+    fake_context = FakeAuroraCommandContext()
+    fake_context.set_options(mock_options)
+    fake_context.add_config(config)
+
+    mock_api = fake_context.get_api('test')
+    fake_context.add_expected_query_result(AuroraClientCommandTest.create_empty_task_result())
+    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, None, config=config)]
+
 
 class TestClientKillCommand(AuroraClientCommandTest):
   @classmethod
@@ -181,9 +248,9 @@ class TestClientKillCommand(AuroraClientCommandTest):
   @classmethod
   def assert_kill_calls(cls, api, instance_range=None, instances=None):
     if instances:
-      kill_calls = [call(AuroraJobKey.from_path(cls.TEST_JOBSPEC), instances)]
+      kill_calls = [call(cls.TEST_JOBKEY, instances, config=None)]
     else:
-      kill_calls = [call(AuroraJobKey.from_path(cls.TEST_JOBSPEC), [i]) for i in instance_range]
+      kill_calls = [call(cls.TEST_JOBKEY, [i], config=None) for i in instance_range]
     assert api.kill_job.mock_calls == kill_calls
 
   @classmethod
@@ -196,7 +263,7 @@ class TestClientKillCommand(AuroraClientCommandTest):
 
   @classmethod
   def assert_kill_call_no_instances(cls, api):
-    assert api.kill_job.mock_calls == call((AuroraJobKey.from_path(cls.TEST_JOBSPEC), None))
+    assert api.kill_job.mock_calls == [call(cls.TEST_JOBKEY, None, config=None)]
 
   @classmethod
   def assert_query(cls, fake_api):
@@ -213,12 +280,8 @@ class TestClientKillCommand(AuroraClientCommandTest):
 
       api = mock_context.get_api('west')
       api.kill_job.return_value = self.create_simple_success_response()
-
-      with temporary_file() as fp:
-        fp.write(self.get_valid_config())
-        fp.flush()
-        cmd = AuroraCommandLine()
-        cmd.execute(['job', 'killall', '--no-batching', '--config=%s' % fp.name, self.TEST_JOBSPEC])
+      cmd = AuroraCommandLine()
+      cmd.execute(['job', 'killall', '--no-batching', self.TEST_JOBSPEC])
 
       self.assert_kill_call_no_instances(api)
       assert mock_monitor.wait_until.mock_calls == [
@@ -237,11 +300,8 @@ class TestClientKillCommand(AuroraClientCommandTest):
       mock_context.add_expected_query_result(
           self.create_query_call_result(), job_key=self.TEST_JOBKEY)
 
-      with temporary_file() as fp:
-        fp.write(self.get_valid_config())
-        fp.flush()
-        cmd = AuroraCommandLine()
-        cmd.execute(['job', 'killall', '--config=%s' % fp.name, self.TEST_JOBSPEC])
+      cmd = AuroraCommandLine()
+      cmd.execute(['job', 'killall', self.TEST_JOBSPEC])
 
       self.assert_kill_calls(api, instance_range=range(20))
       self.assert_wait_calls(mock_monitor, m.terminal, instance_range=range(20))
@@ -256,13 +316,8 @@ class TestClientKillCommand(AuroraClientCommandTest):
         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()
-
-      with temporary_file() as fp:
-        fp.write(self.get_valid_config())
-        fp.flush()
-        cmd = AuroraCommandLine()
-        cmd.execute(['job', 'kill', '--config=%s' % fp.name, '--no-batching',
-            self.get_instance_spec('0,2,4-6')])
+      cmd = AuroraCommandLine()
+      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)
@@ -275,13 +330,9 @@ class TestClientKillCommand(AuroraClientCommandTest):
         patch('apache.aurora.client.cli.jobs.Job.create_context', return_value=mock_context)):
       api = mock_context.get_api('west')
       api.kill_job.return_value = self.create_simple_success_response()
-
-      with temporary_file() as fp:
-        fp.write(self.get_valid_config())
-        fp.flush()
-        cmd = AuroraCommandLine()
-        cmd.execute(['job', 'kill', '--config=%s' % fp.name, '--no-batching', '--strict',
-             self.get_instance_spec('0,2,4-6,11-20')])
+      cmd = AuroraCommandLine()
+      cmd.execute(['job', 'kill', '--no-batching', '--strict',
+           self.get_instance_spec('0,2,4-6,11-20')])
 
       assert api.kill_job.call_count == 0
 
@@ -294,14 +345,8 @@ class TestClientKillCommand(AuroraClientCommandTest):
         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()
-
-      with temporary_file() as fp:
-        fp.write(self.get_valid_config())
-        fp.flush()
-        cmd = AuroraCommandLine()
-        cmd.execute(['job', 'kill', '--config=%s' % fp.name, '--no-batching',
-             self.get_instance_spec('0,2,4-6,11-13')])
-
+      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_wait_calls(mock_monitor, m.terminal, instances=instances)
@@ -318,12 +363,8 @@ class TestClientKillCommand(AuroraClientCommandTest):
           self.create_query_call_result(), job_key=self.TEST_JOBKEY)
 
       api.kill_job.return_value = self.create_simple_success_response()
-
-      with temporary_file() as fp:
-        fp.write(self.get_valid_config())
-        fp.flush()
-        cmd = AuroraCommandLine()
-        cmd.execute(['job', 'kill', '--config=%s' % fp.name, self.get_instance_spec('0-6')])
+      cmd = AuroraCommandLine()
+      cmd.execute(['job', 'kill', self.get_instance_spec('0-6')])
 
       self.assert_kill_calls(api, instance_range=range(7))
       self.assert_wait_calls(mock_monitor, m.terminal, instance_range=range(7))
@@ -341,13 +382,8 @@ class TestClientKillCommand(AuroraClientCommandTest):
           self.create_query_call_result(), job_key=self.TEST_JOBKEY)
 
       api.kill_job.return_value = self.create_simple_success_response()
-
-      with temporary_file() as fp:
-        fp.write(self.get_valid_config())
-        fp.flush()
-        cmd = AuroraCommandLine()
-        cmd.execute(['job', 'kill', '--max-total-failures=1', '--config=%s' % fp.name,
-            self.get_instance_spec('0-4')])
+      cmd = AuroraCommandLine()
+      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))
@@ -364,12 +400,8 @@ class TestClientKillCommand(AuroraClientCommandTest):
       status_response.result = Result(scheduleStatusResult=ScheduleStatusResult(tasks=[]))
       mock_context.add_expected_query_result(status_response)
       api.kill_job.return_value = self.create_simple_success_response()
-
-      with temporary_file() as fp:
-        fp.write(self.get_valid_config())
-        fp.flush()
-        cmd = AuroraCommandLine()
-        cmd.execute(['job', 'kill', '--config=%s' % fp.name, self.get_instance_spec('0,2,4-13')])
+      cmd = AuroraCommandLine()
+      cmd.execute(['job', 'kill', self.get_instance_spec('0,2,4-13')])
 
       assert api.kill_job.call_count == 0
 
@@ -382,11 +414,8 @@ class TestClientKillCommand(AuroraClientCommandTest):
       api = mock_context.get_api('west')
       api.kill_job.return_value = self.create_simple_success_response()
 
-      with temporary_file() as fp:
-        fp.write(self.get_valid_config())
-        fp.flush()
-        cmd = AuroraCommandLine()
-        cmd.execute(['job', 'killall', '--no-batching', '--config=%s' % fp.name, self.TEST_JOBSPEC])
+      cmd = AuroraCommandLine()
+      cmd.execute(['job', 'killall', '--no-batching', self.TEST_JOBSPEC])
 
       assert mock_context.get_out() == ['Job killall succeeded']
       assert mock_context.get_err() == []
@@ -400,13 +429,8 @@ class TestClientKillCommand(AuroraClientCommandTest):
       api = mock_context.get_api('west')
       mock_context.add_expected_query_result(self.create_query_call_result())
       api.kill_job.return_value = self.create_simple_success_response()
-
-      with temporary_file() as fp:
-        fp.write(self.get_valid_config())
-        fp.flush()
-        cmd = AuroraCommandLine()
-        cmd.execute(['job', 'kill', '--config=%s' % fp.name, '--batch-size=5',
-                     self.get_instance_spec('0,2,4-6')])
+      cmd = AuroraCommandLine()
+      cmd.execute(['job', 'kill', '--batch-size=5', self.get_instance_spec('0,2,4-6')])
 
     assert mock_context.get_out() == ['Successfully killed instances [0, 2, 4, 5, 6]',
         'Job kill succeeded']
@@ -422,13 +446,9 @@ class TestClientKillCommand(AuroraClientCommandTest):
       api = mock_context.get_api('west')
       mock_context.add_expected_query_result(self.create_query_call_result())
       api.kill_job.return_value = self.create_simple_success_response()
-
-      with temporary_file() as fp:
-        fp.write(self.get_valid_config())
-        fp.flush()
-        cmd = AuroraCommandLine()
-        cmd.execute(['job', 'kill', '--max-total-failures=1', '--config=%s' % fp.name,
-                     '--batch-size=5', self.get_instance_spec('0,2,4-13')])
+      cmd = AuroraCommandLine()
+      cmd.execute(['job', 'kill', '--max-total-failures=1', '--batch-size=5',
+          self.get_instance_spec('0,2,4-13')])
 
       assert mock_context.get_out() == []
       assert mock_context.get_err() == [

http://git-wip-us.apache.org/repos/asf/aurora/blob/a2e952b9/src/test/python/apache/aurora/client/cli/util.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/util.py b/src/test/python/apache/aurora/client/cli/util.py
index 1fbf050..053c9a5 100644
--- a/src/test/python/apache/aurora/client/cli/util.py
+++ b/src/test/python/apache/aurora/client/cli/util.py
@@ -66,6 +66,7 @@ class FakeAuroraCommandContext(AuroraCommandContext):
     self.task_result = []
     self.out = []
     self.err = []
+    self.config = None
 
   def get_api(self, cluster):
     return self.fake_api
@@ -89,6 +90,12 @@ class FakeAuroraCommandContext(AuroraCommandContext):
     indent_str = " " * indent
     self.err.append("%s%s" % (indent_str, msg))
 
+  def get_job_config(self, jobkey, config_file):
+    if not self.config:
+      return super(FakeAuroraCommandContext, self).get_job_config(jobkey, config_file)
+    else:
+      return self.config
+
   def get_out(self):
     return self.out
 
@@ -114,6 +121,9 @@ class FakeAuroraCommandContext(AuroraCommandContext):
     # each call adds an expected query result, in order.
     self.fake_api.scheduler_proxy.getTasksWithoutConfigs.side_effect = self.task_result
 
+  def add_config(self, config):
+    self.config = config
+
 
 class AuroraClientCommandTest(unittest.TestCase):
   FAKE_TIME = 42131