You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by mc...@apache.org on 2014/03/21 21:05:11 UTC

git commit: Add support for global hooks to the client.

Repository: incubator-aurora
Updated Branches:
  refs/heads/master 79a6043ef -> 8e54fd40b


Add support for global hooks to the client.

To support site-specific policy requirements, we'd like to be able to
have hooks that are applied to every configuration, and that are used
regardless of whether a configuration specifically enables hooks. We'd
like these to be baked in to the client.

This change adds a global registry, which can be initialized with a
set of global hooks. In case of errors/emergencies, there is an
override flag to disable the global hooks.

Bugs closed: aurora-269

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


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

Branch: refs/heads/master
Commit: 8e54fd40b8f1d6567cce7ecddbcfeeaa58683a61
Parents: 79a6043
Author: Mark Chu-Carroll <mc...@twopensource.com>
Authored: Fri Mar 21 16:01:59 2014 -0400
Committer: Mark Chu-Carroll <mc...@twitter.com>
Committed: Fri Mar 21 16:01:59 2014 -0400

----------------------------------------------------------------------
 .../apache/aurora/client/commands/core.py       |  27 +-
 src/main/python/apache/aurora/client/config.py  |  35 +++
 .../apache/aurora/client/hooks/hooked_api.py    |   4 +-
 src/main/python/apache/aurora/client/options.py |   8 +
 .../python/apache/aurora/client/commands/BUILD  |  12 +
 .../aurora/client/commands/test_admin_sla.py    |   3 +-
 .../client/commands/test_cancel_update.py       |   1 +
 .../aurora/client/commands/test_create.py       |   1 +
 .../apache/aurora/client/commands/test_diff.py  |   1 +
 .../apache/aurora/client/commands/test_hooks.py | 247 +++++++++++++++++++
 .../apache/aurora/client/commands/test_kill.py  |   1 +
 .../aurora/client/commands/test_listjobs.py     |   1 +
 .../aurora/client/commands/test_maintenance.py  |   1 +
 .../aurora/client/commands/test_restart.py      |   1 +
 .../apache/aurora/client/commands/test_run.py   |   1 +
 .../apache/aurora/client/commands/test_ssh.py   |   1 +
 .../aurora/client/commands/test_status.py       |   1 +
 .../aurora/client/commands/test_update.py       |   1 +
 .../python/apache/aurora/client/hooks/BUILD     |   2 +
 19 files changed, 344 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8e54fd40/src/main/python/apache/aurora/client/commands/core.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/commands/core.py b/src/main/python/apache/aurora/client/commands/core.py
index 9977c72..7a48ac1 100644
--- a/src/main/python/apache/aurora/client/commands/core.py
+++ b/src/main/python/apache/aurora/client/commands/core.py
@@ -40,12 +40,13 @@ from apache.aurora.client.api.disambiguator import LiveJobDisambiguator
 from apache.aurora.client.api.job_monitor import JobMonitor
 from apache.aurora.client.api.quota_check import print_quota
 from apache.aurora.client.api.updater_util import UpdaterConfig
-from apache.aurora.client.config import get_config
+from apache.aurora.client.config import get_config, GlobalHookRegistry
 from apache.aurora.client.factory import make_client, make_client_factory
 from apache.aurora.client.options import (
     CLUSTER_CONFIG_OPTION,
     CLUSTER_INVOKE_OPTION,
     CLUSTER_NAME_OPTION,
+    DISABLE_HOOKS_OPTION,
     ENV_CONFIG_OPTION,
     ENVIRONMENT_BIND_OPTION,
     FROM_JOBKEY_OPTION,
@@ -111,6 +112,16 @@ def version(args):
   print("Aurora API version: %s" % CURRENT_API_VERSION)
 
 
+def maybe_disable_hooks(options):
+  """Checks the hooks disable option, and disables the hooks if required.
+  This could be done with a callback in the option, but this is better for the way that
+  we test clientv1.
+  """
+  if options.disable_all_hooks_reason is not None:
+    GlobalHookRegistry.disable_hooks()
+    log.info('Client hooks disabled; reason given by user: %s' % options.disable_all_hooks_reason)
+
+
 @app.command
 @app.command_option(ENVIRONMENT_BIND_OPTION)
 @app.command_option(OPEN_BROWSER_OPTION)
@@ -118,6 +129,7 @@ def version(args):
 @app.command_option(ENV_CONFIG_OPTION)
 @app.command_option(JSON_OPTION)
 @app.command_option(WAIT_UNTIL_OPTION)
+@app.command_option(DISABLE_HOOKS_OPTION)
 @requires.exactly('cluster/role/env/job', 'config')
 def create(job_spec, config_file):
   """usage: create cluster/role/env/job config
@@ -125,6 +137,7 @@ def create(job_spec, config_file):
   Creates a job based on a configuration file.
   """
   options = app.get_options()
+  maybe_disable_hooks(options)
   try:
     config = get_job_config(job_spec, config_file, options)
   except ValueError as v:
@@ -305,13 +318,14 @@ def inspect(job_spec, config_file):
 @app.command
 @app.command_option(CLUSTER_INVOKE_OPTION)
 @app.command_option(OPEN_BROWSER_OPTION)
+@app.command_option(DISABLE_HOOKS_OPTION)
 def start_cron(args, options):
   """usage: start_cron cluster/role/env/job
 
   Invokes a cron job immediately, out of its normal cron cycle.
   This does not affect the cron cycle in any way.
   """
-
+  maybe_disable_hooks(options)
   api, job_key, config_file = LiveJobDisambiguator.disambiguate_args_or_die(
       args, options, make_client_factory())
   config = get_job_config(job_key.to_path(), config_file, options) if config_file else None
@@ -374,12 +388,14 @@ def list_jobs(cluster_and_role):
 @app.command_option(CLUSTER_INVOKE_OPTION)
 @app.command_option(OPEN_BROWSER_OPTION)
 @app.command_option(SHARDS_OPTION)
+@app.command_option(DISABLE_HOOKS_OPTION)
 def kill(args, options):
   """usage: kill --shards=shardspec cluster/role/env/job
 
   Kills a group of tasks in a running job, blocking until all specified tasks have terminated.
 
   """
+  maybe_disable_hooks(options)
   if options.shards is None:
     print('Shards option is required for kill; use killall to kill all shards', file=sys.stderr)
     exit(1)
@@ -394,11 +410,12 @@ def kill(args, options):
 @app.command
 @app.command_option(CLUSTER_INVOKE_OPTION)
 @app.command_option(OPEN_BROWSER_OPTION)
+@app.command_option(DISABLE_HOOKS_OPTION)
 def killall(args, options):
   """usage: killall cluster/role/env/job
   Kills all tasks in a running job, blocking until all specified tasks have been terminated.
   """
-
+  maybe_disable_hooks(options)
   job_key = AuroraJobKey.from_path(args[0])
   config_file = args[1] if len(args) > 1 else None  # the config for hooks
   config = get_job_config(job_key.to_path(), config_file, options) if config_file else None
@@ -482,6 +499,7 @@ def status(args, options):
 @app.command_option(ENV_CONFIG_OPTION)
 @app.command_option(JSON_OPTION)
 @app.command_option(HEALTH_CHECK_INTERVAL_SECONDS_OPTION)
+@app.command_option(DISABLE_HOOKS_OPTION)
 @app.command_option(
     '--force',
     dest='force',
@@ -526,6 +544,7 @@ def update(job_spec, config_file):
       time.sleep(5)
 
   options = app.get_options()
+  maybe_disable_hooks(options)
   config = get_job_config(job_spec, config_file, options)
   api = make_client(config.cluster())
   if not options.force:
@@ -572,6 +591,7 @@ def update(job_spec, config_file):
     default=30,
     help='Minimum number of seconds a shard must remain in RUNNING state before considered a '
          'success.')
+@app.command_option(DISABLE_HOOKS_OPTION)
 def restart(args, options):
   """usage: restart cluster/role/env/job
                [--shards=SHARDS]
@@ -586,6 +606,7 @@ def restart(args, options):
 
   Restarts are fully controlled client-side, so aborting halts the restart.
   """
+  maybe_disable_hooks(options)
   api, job_key, config_file = LiveJobDisambiguator.disambiguate_args_or_die(
       args, options, make_client_factory())
   config = get_job_config(job_key.to_path(), config_file, options) if config_file else None

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8e54fd40/src/main/python/apache/aurora/client/config.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/config.py b/src/main/python/apache/aurora/client/config.py
index 1fe5db4..855f709 100644
--- a/src/main/python/apache/aurora/client/config.py
+++ b/src/main/python/apache/aurora/client/config.py
@@ -181,6 +181,41 @@ def populate_namespaces(config, env=None):
   return config
 
 
+class GlobalHookRegistry(object):
+  """To allow enforcable policy, we need a set of mandatory hooks that are
+  registered as part of the client executable. Global hooks are registered
+  by calling GlobalHookRegistry.register_global_hook.
+  """
+
+  HOOKS = []
+
+  DISABLED = False
+
+  @classmethod
+  def register_global_hook(cls, hook):
+    cls.HOOKS.append(hook)
+
+  @classmethod
+  def get_hooks(cls):
+    if cls.DISABLED:
+      return []
+    else:
+      return cls.HOOKS[:]
+
+  @classmethod
+  def reset(cls):
+    cls.HOOKS = []
+    cls.DISABLED = False
+
+  @classmethod
+  def disable_hooks(cls):
+    cls.DISABLED = True
+
+  @classmethod
+  def enable_hooks(cls):
+    cls.DISABLED = False
+
+
 def inject_hooks(config, env=None):
   config.hooks = (env or {}).get('hooks', [])
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8e54fd40/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 6de8134..48e8514 100644
--- a/src/main/python/apache/aurora/client/hooks/hooked_api.py
+++ b/src/main/python/apache/aurora/client/hooks/hooked_api.py
@@ -18,6 +18,7 @@ import functools
 import traceback
 
 from apache.aurora.client.api import AuroraClientAPI
+from apache.aurora.client.config import GlobalHookRegistry
 from apache.aurora.common.aurora_job_key import AuroraJobKey
 
 from gen.apache.aurora.ttypes import ResponseCode
@@ -126,7 +127,8 @@ class HookedAuroraClientAPI(NonHookedAuroraClientAPI):
 
   @classmethod
   def _yield_hooks(cls, event, config, job_key, api_call, extra_argument=None):
-    hooks = config.hooks if config and config.raw().enable_hooks().get() else ()
+    hooks = GlobalHookRegistry.get_hooks()
+    hooks += (config.hooks if config and config.raw().enable_hooks().get() else [])
     for hook in hooks:
       yield cls._meta_hook(hook,
           cls._generate_method(hook, config, job_key, event, api_call, extra_argument))

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8e54fd40/src/main/python/apache/aurora/client/options.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/options.py b/src/main/python/apache/aurora/client/options.py
index b8278bd..0d85c36 100644
--- a/src/main/python/apache/aurora/client/options.py
+++ b/src/main/python/apache/aurora/client/options.py
@@ -24,6 +24,7 @@ __all__ = (
   'CLUSTER_CONFIG_OPTION',
   'CLUSTER_INVOKE_OPTION',
   'CLUSTER_NAME_OPTION',
+  'DISABLE_HOOKS_OPTION',
   'ENVIRONMENT_BIND_OPTION',
   'ENV_CONFIG_OPTION',
   'EXECUTOR_SANDBOX_OPTION',
@@ -102,6 +103,13 @@ def make_env_option(explanation):
 # Note: in these predefined options, "OPTION" is used in names of optional arguments,
 # and "PARAMETER" is used in names of required ones.
 
+DISABLE_HOOKS_OPTION = optparse.Option(
+    '--disable_all_hooks_reason',
+    type=str,
+    default=None,
+    help='Override standard behavior and disable the builtin hooks')
+
+
 OPEN_BROWSER_OPTION = optparse.Option(
     '-o',
     '--open_browser',

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8e54fd40/src/test/python/apache/aurora/client/commands/BUILD
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/commands/BUILD b/src/test/python/apache/aurora/client/commands/BUILD
index 7c80808..a7845ae 100644
--- a/src/test/python/apache/aurora/client/commands/BUILD
+++ b/src/test/python/apache/aurora/client/commands/BUILD
@@ -19,6 +19,7 @@ python_test_suite(
   dependencies = [
     pants(':admin'),
     pants(':core'),
+    pants(':hooks'),
     pants(':maintenance'),
     pants(':run'),
     pants(':ssh'),
@@ -45,6 +46,17 @@ python_tests(
   ])
 
 python_tests(
+  name='hooks',
+  sources = [ 'test_hooks.py' ],
+  dependencies = [
+    pants(':util'),
+    pants('3rdparty/python:mock'),
+    pants('3rdparty/python:twitter.common.contextutil'),
+    pants('src/main/python/apache/aurora/client/commands:core'),
+    pants('src/main/thrift/org/apache/aurora/gen:py-thrift'),
+  ])
+
+python_tests(
   name='admin',
   sources = [
     'test_admin_sla.py',

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8e54fd40/src/test/python/apache/aurora/client/commands/test_admin_sla.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/commands/test_admin_sla.py b/src/test/python/apache/aurora/client/commands/test_admin_sla.py
index 51e7482..2f09cb1 100644
--- a/src/test/python/apache/aurora/client/commands/test_admin_sla.py
+++ b/src/test/python/apache/aurora/client/commands/test_admin_sla.py
@@ -42,6 +42,7 @@ class TestAdminSlaListSafeDomainCommand(AuroraClientCommandTest):
     mock_options.override_filename = override
     mock_options.list_jobs = list_jobs
     mock_options.verbosity = False
+    mock_options.disable_all_hooks = False
     return mock_options
 
   @classmethod
@@ -360,4 +361,4 @@ class TestAdminSlaProbeHostsCommand(AuroraClientCommandTest):
         except SystemExit:
           pass
         else:
-          assert 'Expected error is not raised.'
\ No newline at end of file
+          assert 'Expected error is not raised.'

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8e54fd40/src/test/python/apache/aurora/client/commands/test_cancel_update.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/commands/test_cancel_update.py b/src/test/python/apache/aurora/client/commands/test_cancel_update.py
index 7824b76..da9ed2f 100644
--- a/src/test/python/apache/aurora/client/commands/test_cancel_update.py
+++ b/src/test/python/apache/aurora/client/commands/test_cancel_update.py
@@ -46,6 +46,7 @@ class TestClientCancelUpdateCommand(AuroraClientCommandTest):
     mock_options.shards = None
     mock_options.cluster = None
     mock_options.json = False
+    mock_options.disable_all_hooks = False
     return mock_options
 
   @classmethod

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8e54fd40/src/test/python/apache/aurora/client/commands/test_create.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/commands/test_create.py b/src/test/python/apache/aurora/client/commands/test_create.py
index adbff90..ceea25d 100644
--- a/src/test/python/apache/aurora/client/commands/test_create.py
+++ b/src/test/python/apache/aurora/client/commands/test_create.py
@@ -52,6 +52,7 @@ class TestClientCreateCommand(AuroraClientCommandTest):
     mock_options.open_browser = False
     mock_options.cluster = None
     mock_options.wait_until = 'RUNNING'  # or 'FINISHED' for other tests
+    mock_options.disable_all_hooks = False
     return mock_options
 
   @classmethod

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8e54fd40/src/test/python/apache/aurora/client/commands/test_diff.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/commands/test_diff.py b/src/test/python/apache/aurora/client/commands/test_diff.py
index fc15421..2412d09 100644
--- a/src/test/python/apache/aurora/client/commands/test_diff.py
+++ b/src/test/python/apache/aurora/client/commands/test_diff.py
@@ -53,6 +53,7 @@ class TestDiffCommand(AuroraClientCommandTest):
     mock_options.open_browser = False
     mock_options.rename_from = None
     mock_options.cluster = None
+    mock_options.disable_all_hooks = False
     return mock_options
 
   @classmethod

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8e54fd40/src/test/python/apache/aurora/client/commands/test_hooks.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/commands/test_hooks.py b/src/test/python/apache/aurora/client/commands/test_hooks.py
new file mode 100644
index 0000000..64afce5
--- /dev/null
+++ b/src/test/python/apache/aurora/client/commands/test_hooks.py
@@ -0,0 +1,247 @@
+#
+# Copyright 2013 Apache Software Foundation
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+import contextlib
+import unittest
+
+from apache.aurora.client.commands.core import create
+from apache.aurora.client.commands.util import AuroraClientCommandTest
+from apache.aurora.client.config import AuroraConfig, GlobalHookRegistry
+from apache.aurora.client.hooks.hooked_api import HookedAuroraClientAPI
+from twitter.common import app
+from twitter.common.contextutil import temporary_file
+
+from gen.apache.aurora.ttypes import (
+    AssignedTask,
+    Identity,
+    Response,
+    ResponseCode,
+    Result,
+    ScheduledTask,
+    ScheduleStatus,
+    ScheduleStatusResult,
+    TaskEvent,
+    TaskQuery,
+)
+
+from mock import Mock, patch
+from pystachio.config import Config
+
+
+class CreateHookForTesting(object):
+  def __init__(self, succeed):
+    self.created_jobs = []
+    self.succeed = succeed
+
+  def pre_create_job(self, api, config):
+    self.created_jobs.append(config)
+    return self.succeed
+
+
+class TestClientCreateCommand(AuroraClientCommandTest):
+
+  def setUp(self):
+    GlobalHookRegistry.reset()
+
+  @classmethod
+  def setup_mock_options(cls):
+    """set up to get a mock options object."""
+    mock_options = Mock()
+    mock_options.json = False
+    mock_options.bindings = {}
+    mock_options.open_browser = False
+    mock_options.cluster = None
+    mock_options.wait_until = 'RUNNING'  # or 'FINISHED' for other tests
+    mock_options.disable_all_hooks_reason = None
+    return mock_options
+
+  @classmethod
+  def create_mock_task(cls, task_id, instance_id, initial_time, status):
+    mock_task = Mock(spec=ScheduledTask)
+    mock_task.assignedTask = Mock(spec=AssignedTask)
+    mock_task.assignedTask.taskId = task_id
+    mock_task.assignedTask.instanceId = instance_id
+    mock_task.status = status
+    mock_task_event = Mock(spec=TaskEvent)
+    mock_task_event.timestamp = initial_time
+    mock_task.taskEvents = [mock_task_event]
+    return mock_task
+
+  @classmethod
+  def create_mock_status_query_result(cls, scheduleStatus):
+    mock_query_result = cls.create_simple_success_response()
+    mock_query_result.result.scheduleStatusResult = Mock(spec=ScheduleStatusResult)
+    if scheduleStatus == ScheduleStatus.INIT:
+      # status query result for before job is launched.
+      mock_query_result.result.scheduleStatusResult.tasks = []
+    else:
+      mock_task_one = cls.create_mock_task('hello', 0, 1000, scheduleStatus)
+      mock_task_two = cls.create_mock_task('hello', 1, 1004, scheduleStatus)
+      mock_query_result.result.scheduleStatusResult.tasks = [mock_task_one, mock_task_two]
+    return mock_query_result
+
+  @classmethod
+  def create_mock_query(cls):
+    return TaskQuery(owner=Identity(role=cls.TEST_ROLE), environment=cls.TEST_ENV,
+        jobName=cls.TEST_JOB)
+
+  @classmethod
+  def get_createjob_response(cls):
+    # Then, we call api.create_job(config)
+    return cls.create_simple_success_response()
+
+  @classmethod
+  def get_failed_createjob_response(cls):
+    return cls.create_error_response()
+
+  @classmethod
+  def assert_create_job_called(cls, mock_api):
+    # Check that create_job was called exactly once, with an AuroraConfig parameter.
+    assert mock_api.create_job.call_count == 1
+    assert isinstance(mock_api.create_job.call_args_list[0][0][0], AuroraConfig)
+
+  @classmethod
+  def assert_scheduler_called(cls, mock_api, mock_query, num_queries):
+    # scheduler.scheduler() is called once, as a part of the handle_open call.
+    assert mock_api.scheduler_proxy.getTasksStatus.call_count == num_queries
+    mock_api.scheduler_proxy.getTasksStatus.assert_called_with(mock_query)
+
+  def test_create_job_hook_called(self):
+    """Run a test of the "create" command against a mocked API;
+    verifies that a required hook runs, even though the config doesn't mention it.
+    """
+    # Create a hook on "create_job" that just adds something to a list in the test.
+    # Patch in HookedAuroraClientAPI to replace the UnhookedAuroraClientAPI with a mock.
+
+    mock_options = self.setup_mock_options()
+    hook = CreateHookForTesting(True)
+    GlobalHookRegistry.register_global_hook(hook)
+
+    # create first calls get_job_config, which calls get_config. As long as we've got the options
+    # set up correctly, this should work.
+
+    # Next, create gets an API object via make_client. We need to replace that with a mock API.
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
+    with contextlib.nested(
+        patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS),
+        patch('apache.aurora.client.api.SchedulerProxy', return_value = mock_scheduler_proxy),
+        patch('twitter.common.app.get_options', return_value=mock_options)):
+
+      mock_query = self.create_mock_query()
+      mock_scheduler_proxy.createJob.return_value=self.get_createjob_response()
+
+      mock_scheduler_proxy.getTasksStatus.side_effect = [
+        self.create_mock_status_query_result(ScheduleStatus.INIT),
+        self.create_mock_status_query_result(ScheduleStatus.RUNNING)
+      ]
+      # Finally, it calls the monitor to watch and make sure the jobs started;
+      # but we already set that up in the side-effects list for the query mock.
+
+      # This is the real test: invoke create as if it had been called by the command line.
+      with temporary_file() as fp:
+        fp.write(self.get_valid_config())
+        fp.flush()
+        create(['west/mchucarroll/test/hello', fp.name])
+
+      # Now check that the right API calls got made.
+      assert mock_scheduler_proxy.createJob.call_count == 1
+      assert len(hook.created_jobs) == 1
+
+  def test_create_job_hook_aborts(self):
+    """Run a test of the "create" command against a mocked API;
+    verifies that a required hook runs, even though the config doesn't mention it.
+    """
+    # Create a hook on "create_job" that just adds something to a list in the test.
+    # Patch in HookedAuroraClientAPI to replace the UnhookedAuroraClientAPI with a mock.
+    mock_options = self.setup_mock_options()
+    hook = CreateHookForTesting(False)
+    GlobalHookRegistry.register_global_hook(hook)
+
+    # create first calls get_job_config, which calls get_config. As long as we've got the options
+    # set up correctly, this should work.
+
+    # Next, create gets an API object via make_client. We need to replace that with a mock API.
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
+    with contextlib.nested(
+        patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS),
+        patch('apache.aurora.client.api.SchedulerProxy', return_value = mock_scheduler_proxy),
+        patch('twitter.common.app.get_options', return_value=mock_options)):
+
+      mock_query = self.create_mock_query()
+      mock_scheduler_proxy.createJob.return_value=self.get_createjob_response()
+
+      mock_scheduler_proxy.getTasksStatus.side_effect = [
+        self.create_mock_status_query_result(ScheduleStatus.INIT),
+        self.create_mock_status_query_result(ScheduleStatus.RUNNING)
+      ]
+
+      # Finally, it calls the monitor to watch and make sure the jobs started;
+      # but we already set that up in the side-effects list for the query mock.
+
+      # This is the real test: invoke create as if it had been called by the command line.
+      with temporary_file() as fp:
+        fp.write(self.get_valid_config())
+        fp.flush()
+        self.assertRaises(HookedAuroraClientAPI.PreHooksStoppedCall, create,
+            ['west/mchucarroll/test/hello', fp.name])
+
+      # Now check that the right API calls got made.
+      assert mock_scheduler_proxy.createJob.call_count == 0
+      assert len(hook.created_jobs) == 1
+
+
+  def test_block_hooks(self):
+    """Run a test of the "create" command against a mocked API;
+    verifies that a required hook runs, even though the config doesn't mention it.
+    """
+    # Create a hook on "create_job" that just adds something to a list in the test.
+    # Patch in HookedAuroraClientAPI to replace the UnhookedAuroraClientAPI with a mock.
+
+    mock_options = self.setup_mock_options()
+    hook = CreateHookForTesting(True)
+    GlobalHookRegistry.register_global_hook(hook)
+    mock_options.disable_all_hooks_reason = "Because I said so."
+
+
+    # create first calls get_job_config, which calls get_config. As long as we've got the options
+    # set up correctly, this should work.
+
+    # Next, create gets an API object via make_client. We need to replace that with a mock API.
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
+    with contextlib.nested(
+        patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS),
+        patch('apache.aurora.client.api.SchedulerProxy', return_value = mock_scheduler_proxy),
+        patch('twitter.common.app.get_options', return_value=mock_options)):
+
+      mock_query = self.create_mock_query()
+      mock_scheduler_proxy.createJob.return_value=self.get_createjob_response()
+
+      mock_scheduler_proxy.getTasksStatus.side_effect = [
+        self.create_mock_status_query_result(ScheduleStatus.INIT),
+        self.create_mock_status_query_result(ScheduleStatus.RUNNING)
+      ]
+      # Finally, it calls the monitor to watch and make sure the jobs started;
+      # but we already set that up in the side-effects list for the query mock.
+
+      # This is the real test: invoke create as if it had been called by the command line.
+      with temporary_file() as fp:
+        fp.write(self.get_valid_config())
+        fp.flush()
+        create(['west/mchucarroll/test/hello', fp.name])
+
+      # Now check that the right API calls got made.
+      assert mock_scheduler_proxy.createJob.call_count == 1
+      assert len(hook.created_jobs) == 0

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8e54fd40/src/test/python/apache/aurora/client/commands/test_kill.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/commands/test_kill.py b/src/test/python/apache/aurora/client/commands/test_kill.py
index 75e754b..825f5f1 100644
--- a/src/test/python/apache/aurora/client/commands/test_kill.py
+++ b/src/test/python/apache/aurora/client/commands/test_kill.py
@@ -45,6 +45,7 @@ class TestClientKillCommand(AuroraClientCommandTest):
     mock_options.shards = None
     mock_options.cluster = None
     mock_options.json = False
+    mock_options.disable_all_hooks = False
     return mock_options
 
   @classmethod

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8e54fd40/src/test/python/apache/aurora/client/commands/test_listjobs.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/commands/test_listjobs.py b/src/test/python/apache/aurora/client/commands/test_listjobs.py
index 099475c..11a255b 100644
--- a/src/test/python/apache/aurora/client/commands/test_listjobs.py
+++ b/src/test/python/apache/aurora/client/commands/test_listjobs.py
@@ -38,6 +38,7 @@ class TestListJobs(AuroraClientCommandTest):
     mock_options = Mock()
     mock_options.pretty = False
     mock_options.show_cron = False
+    mock_options.disable_all_hooks = False
     return mock_options
 
   @classmethod

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8e54fd40/src/test/python/apache/aurora/client/commands/test_maintenance.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/commands/test_maintenance.py b/src/test/python/apache/aurora/client/commands/test_maintenance.py
index df368e2..0404a98 100644
--- a/src/test/python/apache/aurora/client/commands/test_maintenance.py
+++ b/src/test/python/apache/aurora/client/commands/test_maintenance.py
@@ -46,6 +46,7 @@ class TestMaintenanceCommands(AuroraClientCommandTest):
     mock_options.hosts = ','.join(self.HOSTNAMES)
     mock_options.cluster = self.TEST_CLUSTER
     mock_options.verbosity = False
+    mock_options.disable_all_hooks = False
     return mock_options
 
   def create_host_statuses(self, maintenance_mode):

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8e54fd40/src/test/python/apache/aurora/client/commands/test_restart.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/commands/test_restart.py b/src/test/python/apache/aurora/client/commands/test_restart.py
index 1cab2a2..075414b 100644
--- a/src/test/python/apache/aurora/client/commands/test_restart.py
+++ b/src/test/python/apache/aurora/client/commands/test_restart.py
@@ -53,6 +53,7 @@ class TestRestartCommand(AuroraClientCommandTest):
     mock_options.max_total_failures = 0
     mock_options.restart_threshold = 30
     mock_options.watch_secs = 30
+    mock_options.disable_all_hooks_reason = None
     return mock_options
 
   @classmethod

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8e54fd40/src/test/python/apache/aurora/client/commands/test_run.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/commands/test_run.py b/src/test/python/apache/aurora/client/commands/test_run.py
index fe66615..b808ed2 100644
--- a/src/test/python/apache/aurora/client/commands/test_run.py
+++ b/src/test/python/apache/aurora/client/commands/test_run.py
@@ -45,6 +45,7 @@ class TestRunCommand(AuroraClientCommandTest):
     mock_options.tunnels = []
     mock_options.executor_sandbox = False
     mock_options.ssh_user = None
+    mock_options.disable_all_hooks = False
     return mock_options
 
   @classmethod

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8e54fd40/src/test/python/apache/aurora/client/commands/test_ssh.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/commands/test_ssh.py b/src/test/python/apache/aurora/client/commands/test_ssh.py
index 2fe12ef..656a334 100644
--- a/src/test/python/apache/aurora/client/commands/test_ssh.py
+++ b/src/test/python/apache/aurora/client/commands/test_ssh.py
@@ -44,6 +44,7 @@ class TestSshCommand(AuroraClientCommandTest):
     mock_options.tunnels = []
     mock_options.executor_sandbox = False
     mock_options.ssh_user = None
+    mock_options.disable_all_hooks = False
     return mock_options
 
   @classmethod

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8e54fd40/src/test/python/apache/aurora/client/commands/test_status.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/commands/test_status.py b/src/test/python/apache/aurora/client/commands/test_status.py
index eed798d..a1f200f 100644
--- a/src/test/python/apache/aurora/client/commands/test_status.py
+++ b/src/test/python/apache/aurora/client/commands/test_status.py
@@ -44,6 +44,7 @@ class TestListJobs(AuroraClientCommandTest):
     mock_options = Mock()
     mock_options.pretty = False
     mock_options.show_cron = False
+    mock_options.disable_all_hooks = False
     return mock_options
 
   @classmethod

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8e54fd40/src/test/python/apache/aurora/client/commands/test_update.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/commands/test_update.py b/src/test/python/apache/aurora/client/commands/test_update.py
index 21951b6..72df41b 100644
--- a/src/test/python/apache/aurora/client/commands/test_update.py
+++ b/src/test/python/apache/aurora/client/commands/test_update.py
@@ -63,6 +63,7 @@ class TestUpdateCommand(AuroraClientCommandTest):
     mock_options.env = None
     mock_options.shards = None
     mock_options.health_check_interval_seconds = 3
+    mock_options.disable_all_hooks = False
     return mock_options
 
   @classmethod

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8e54fd40/src/test/python/apache/aurora/client/hooks/BUILD
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/hooks/BUILD b/src/test/python/apache/aurora/client/hooks/BUILD
index 3d6f796..a9b34cf 100644
--- a/src/test/python/apache/aurora/client/hooks/BUILD
+++ b/src/test/python/apache/aurora/client/hooks/BUILD
@@ -28,6 +28,7 @@ python_tests(name = 'hooked_api',
     pants('src/main/python/apache/aurora/common:cluster'),
     pants('src/main/python/apache/aurora/client:api'),
     pants('src/main/python/apache/aurora/client/hooks'),
+    pants('src/main/python/apache/aurora/client:config')
   ]
 )
 
@@ -36,5 +37,6 @@ python_tests(name = 'non_hooked_api',
   dependencies = [
     pants('src/main/python/apache/aurora/common:aurora_job_key'),
     pants('src/main/python/apache/aurora/client/hooks'),
+    pants('src/main/python/apache/aurora/client:config')
   ],
 )