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/01/17 20:54:32 UTC

[1/2] Get rid of ambiguities about exactly what object is being used to send calls to the scheduler. (See AURORA-42).

Updated Branches:
  refs/heads/master b6fbcde05 -> b17bc1f96


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/b17bc1f9/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 de6039b..45ecd12 100644
--- a/src/test/python/apache/aurora/client/commands/test_update.py
+++ b/src/test/python/apache/aurora/client/commands/test_update.py
@@ -58,7 +58,7 @@ class TestUpdateCommand(AuroraClientCommandTest):
   # that the client makes the right API calls.
   def test_update_command_line_succeeds(self):
     mock_options = self.setup_mock_options()
-    (mock_api, mock_scheduler) = self.create_mock_api()
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
     with contextlib.nested(
         patch('apache.aurora.client.commands.core.make_client', return_value=mock_api),
         patch('twitter.common.app.get_options', return_value=mock_options),
@@ -81,7 +81,7 @@ class TestUpdateCommand(AuroraClientCommandTest):
 
   def test_update_invalid_config(self):
     mock_options = self.setup_mock_options()
-    (mock_api, mock_scheduler) = self.create_mock_api()
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
     # Set up the context to capture the make_client and get_options calls.
     with contextlib.nested(
         patch('apache.aurora.client.commands.core.make_client', return_value=mock_api),
@@ -100,7 +100,7 @@ class TestUpdateCommand(AuroraClientCommandTest):
   @classmethod
   def setup_mock_scheduler_for_simple_update(cls, api):
     """Set up all of the API mocks for scheduler calls during a simple update"""
-    sched_proxy = api.scheduler.scheduler
+    sched_proxy = api.scheduler_proxy
     # First, the updater acquires a lock
     sched_proxy.acquireLock.return_value = cls.create_acquire_lock_response(ResponseCode.OK,
          'OK', 'token', False)
@@ -156,9 +156,9 @@ class TestUpdateCommand(AuroraClientCommandTest):
     return release_lock_response
 
   @classmethod
-  def setup_get_tasks_status_calls(cls, scheduler):
+  def setup_get_tasks_status_calls(cls, scheduler_proxy):
     status_response = cls.create_simple_success_response()
-    scheduler.getTasksStatus.return_value = status_response
+    scheduler_proxy.getTasksStatus.return_value = status_response
     schedule_status = Mock(spec=ScheduleStatusResult)
     status_response.result.scheduleStatusResult = schedule_status
     task_config = TaskConfig(numCpus=1.0, ramMb=10, diskMb=1)
@@ -175,9 +175,9 @@ class TestUpdateCommand(AuroraClientCommandTest):
       schedule_status.tasks.append(task_status)
 
   @classmethod
-  def assert_start_update_called(cls, mock_scheduler):
-    assert mock_scheduler.scheduler.startUpdate.call_count == 1
-    assert isinstance(mock_scheduler.scheduler.startUpdate.call_args[0][0], JobConfiguration)
+  def assert_start_update_called(cls, mock_scheduler_proxy):
+    assert mock_scheduler_proxy.startUpdate.call_count == 1
+    assert isinstance(mock_scheduler_proxy.startUpdate.call_args[0][0], JobConfiguration)
 
   @classmethod
   def setup_health_checks(cls, mock_api):
@@ -194,13 +194,13 @@ class TestUpdateCommand(AuroraClientCommandTest):
     # Test the client-side updater logic in its simplest case: everything succeeds, and no rolling
     # updates.
     mock_options = self.setup_mock_options()
-    (mock_api, mock_scheduler) = self.create_mock_api()
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
     mock_health_check = self.setup_health_checks(mock_api)
     mock_quota_check = self.setup_quota_check()
 
     with contextlib.nested(
         patch('twitter.common.app.get_options', return_value=mock_options),
-        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler.scheduler),
+        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
         patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS),
         patch('apache.aurora.client.api.instance_watcher.InstanceWatcherHealthCheck',
             return_value=mock_health_check),
@@ -225,11 +225,11 @@ class TestUpdateCommand(AuroraClientCommandTest):
       # and finally release the lock.
       # The kill/start should happen in rolling batches.
       assert options.call_count == 2
-      assert mock_scheduler.scheduler.acquireLock.call_count == 1
-      self.assert_correct_killtask_calls(mock_scheduler.scheduler)
-      self.assert_correct_addinstance_calls(mock_scheduler.scheduler)
-      self.assert_correct_status_calls(mock_scheduler.scheduler)
-      assert mock_scheduler.scheduler.releaseLock.call_count == 1
+      assert mock_scheduler_proxy.acquireLock.call_count == 1
+      self.assert_correct_killtask_calls(mock_scheduler_proxy)
+      self.assert_correct_addinstance_calls(mock_scheduler_proxy)
+      self.assert_correct_status_calls(mock_scheduler_proxy)
+      assert mock_scheduler_proxy.releaseLock.call_count == 1
 
   @classmethod
   def assert_correct_addinstance_calls(cls, api):

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/b17bc1f9/src/test/python/apache/aurora/client/commands/util.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/commands/util.py b/src/test/python/apache/aurora/client/commands/util.py
index 22694a2..426216c 100644
--- a/src/test/python/apache/aurora/client/commands/util.py
+++ b/src/test/python/apache/aurora/client/commands/util.py
@@ -33,24 +33,27 @@ class AuroraClientCommandTest(unittest.TestCase):
   @classmethod
   def create_mock_api(cls):
     """Builds up a mock API object, with a mock SchedulerProxy"""
+    # This looks strange, but we set up the same object to use as both
+    # the SchedulerProxy and the SchedulerClient. These tests want to observe
+    # what API calls get made against the scheduler, and both of these objects
+    # delegate calls to the scheduler. It doesn't matter which one is used:
+    # what we care about is that the right API calls get made.
     mock_api = Mock(spec=HookedAuroraClientAPI)
-    mock_scheduler = Mock()
-    mock_scheduler.url = "http://something_or_other"
-    mock_scheduler_client = Mock()
-    mock_scheduler_client.scheduler.return_value = mock_scheduler
-    mock_scheduler_client.url = "http://something_or_other"
+    mock_scheduler_proxy = Mock()
+    mock_scheduler_proxy.url = "http://something_or_other"
+    mock_scheduler_proxy.scheduler_client.return_value = mock_scheduler_proxy
     mock_api = Mock(spec=HookedAuroraClientAPI)
-    mock_api.scheduler = mock_scheduler_client
-    return (mock_api, mock_scheduler_client)
+    mock_api.scheduler_proxy = mock_scheduler_proxy
+    return (mock_api, mock_scheduler_proxy)
 
   @classmethod
   def create_mock_api_factory(cls):
     """Create a collection of mocks for a test that wants to mock out the client API
     by patching the api factory."""
-    mock_api, mock_scheduler_client = cls.create_mock_api()
+    mock_api, mock_scheduler_proxy = cls.create_mock_api()
     mock_api_factory = Mock()
     mock_api_factory.return_value = mock_api
-    return mock_api_factory, mock_scheduler_client
+    return mock_api_factory, mock_scheduler_proxy
 
   FAKE_TIME = 42131
 


[2/2] git commit: Get rid of ambiguities about exactly what object is being used to send calls to the scheduler. (See AURORA-42).

Posted by mc...@apache.org.
Get rid of ambiguities about exactly what object is being used to send calls to the scheduler. (See AURORA-42).

Also do some cleanups of sloppy stuff in tests.

Testing Done:
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 3 items

src/test/python/apache/aurora/admin/test_mesos_maintenance.py ...

=========================== 3 passed in 0.18 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 2 items

src/test/python/apache/aurora/client/test_binding_helper.py ..

=========================== 2 passed in 0.19 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 6 items

src/test/python/apache/aurora/client/test_config.py ......

=========================== 6 passed in 0.26 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 6 items

src/test/python/apache/aurora/client/api/test_disambiguator.py ......

=========================== 6 passed in 0.18 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 1 items

src/test/python/apache/aurora/client/api/test_job_monitor.py .

=========================== 1 passed in 0.15 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 6 items

src/test/python/apache/aurora/client/api/test_restarter.py ......

=========================== 6 passed in 0.16 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 47 items / 1 skipped

src/test/python/apache/aurora/client/api/test_scheduler_client.py ...............................................

===================== 47 passed, 1 skipped in 0.53 seconds =====================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 20 items

src/test/python/apache/aurora/client/api/test_instance_watcher.py ........
src/test/python/apache/aurora/client/api/test_health_check.py ............

========================== 20 passed in 0.16 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 26 items

src/test/python/apache/aurora/client/api/test_updater.py ..........................

========================== 26 passed in 0.36 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 6 items

src/test/python/apache/aurora/client/api/test_quota_check.py ......

=========================== 6 passed in 0.06 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 17 items

src/test/python/apache/aurora/client/cli/test_create.py ....
src/test/python/apache/aurora/client/cli/test_kill.py .....
src/test/python/apache/aurora/client/cli/test_status.py .....
src/test/python/apache/aurora/client/cli/test_diff.py ...

========================== 17 passed in 0.48 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 24 items

src/test/python/apache/aurora/client/commands/test_cancel_update.py ..
src/test/python/apache/aurora/client/commands/test_create.py ......
src/test/python/apache/aurora/client/commands/test_diff.py ...
src/test/python/apache/aurora/client/commands/test_kill.py ...
src/test/python/apache/aurora/client/commands/test_listjobs.py ..
src/test/python/apache/aurora/client/commands/test_restart.py ...
src/test/python/apache/aurora/client/commands/test_status.py ..
src/test/python/apache/aurora/client/commands/test_update.py ...

========================== 24 passed in 0.78 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 1 items

src/test/python/apache/aurora/client/commands/test_run.py .

=========================== 1 passed in 0.22 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 1 items

src/test/python/apache/aurora/client/commands/test_ssh.py .

=========================== 1 passed in 0.22 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 12 items

src/test/python/apache/aurora/client/hooks/test_hooked_api.py ............

========================== 12 passed in 0.17 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 4 items

src/test/python/apache/aurora/client/hooks/test_non_hooked_api.py ....

=========================== 4 passed in 0.16 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 1 items

src/test/python/apache/aurora/common/test_aurora_job_key.py .

=========================== 1 passed in 0.03 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 1 items

src/test/python/apache/aurora/common/test_cluster.py .

=========================== 1 passed in 0.02 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 4 items

src/test/python/apache/aurora/common/test_clusters.py ....

=========================== 4 passed in 0.08 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 2 items

src/test/python/apache/aurora/common/test_cluster_option.py ..

=========================== 2 passed in 0.03 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 3 items

src/test/python/apache/aurora/common/test_http_signaler.py ...

=========================== 3 passed in 0.05 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 7 items

src/test/python/apache/aurora/config/test_base.py .......

=========================== 7 passed in 0.70 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 1 items

src/test/python/apache/aurora/config/test_constraint_parsing.py .

=========================== 1 passed in 0.06 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 6 items

src/test/python/apache/aurora/config/test_loader.py ......

=========================== 6 passed in 0.11 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 10 items

src/test/python/apache/aurora/config/test_thrift.py ..........

========================== 10 passed in 0.87 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 4 items

src/test/python/apache/aurora/executor/test_executor_detector.py ....

=========================== 4 passed in 0.04 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 4 items

src/test/python/apache/aurora/executor/test_executor_vars.py ....

=========================== 4 passed in 0.10 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 5 items

src/test/python/apache/aurora/executor/test_thermos_task_runner.py .....

========================== 5 passed in 16.81 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 2 items

src/test/python/apache/aurora/executor/common/test_directory_sandbox.py ..

=========================== 2 passed in 0.04 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 3 items

src/test/python/apache/aurora/executor/common/test_health_checker.py ...

=========================== 3 passed in 1.33 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 1 items

src/test/python/apache/aurora/executor/common/test_status_checker.py .

=========================== 1 passed in 0.04 seconds ===========================
============================= test session starts ==============================
platform darwin -- Python 2.7.5 -- pytest-2.5.1
collected 1 items

src/test/python/apache/aurora/executor/common/test_task_info.py .

=========================== 1 passed in 0.13 seconds ===========================
Build operating on targets: OrderedSet([PythonTestSuite(src/test/python/apache/aurora/BUILD:all)])
src.test.python.apache.aurora.admin.mesos_maintenance                           .....   SUCCESS
src.test.python.apache.aurora.client.api.disambiguator                          .....   SUCCESS
src.test.python.apache.aurora.client.api.instance_watcher                       .....   SUCCESS
src.test.python.apache.aurora.client.api.job_monitor                            .....   SUCCESS
src.test.python.apache.aurora.client.api.quota_check                            .....   SUCCESS
src.test.python.apache.aurora.client.api.restarter                              .....   SUCCESS
src.test.python.apache.aurora.client.api.scheduler_client                       .....   SUCCESS
src.test.python.apache.aurora.client.api.updater                                .....   SUCCESS
src.test.python.apache.aurora.client.binding_helper                             .....   SUCCESS
src.test.python.apache.aurora.client.cli.job                                    .....   SUCCESS
src.test.python.apache.aurora.client.commands.core                              .....   SUCCESS
src.test.python.apache.aurora.client.commands.run                               .....   SUCCESS
src.test.python.apache.aurora.client.commands.ssh                               .....   SUCCESS
src.test.python.apache.aurora.client.config                                     .....   SUCCESS
src.test.python.apache.aurora.client.hooks.hooked_api                           .....   SUCCESS
src.test.python.apache.aurora.client.hooks.non_hooked_api                       .....   SUCCESS
src.test.python.apache.aurora.common.test_aurora_job_key                        .....   SUCCESS
src.test.python.apache.aurora.common.test_cluster                               .....   SUCCESS
src.test.python.apache.aurora.common.test_cluster_option                        .....   SUCCESS
src.test.python.apache.aurora.common.test_clusters                              .....   SUCCESS
src.test.python.apache.aurora.common.test_http_signaler                         .....   SUCCESS
src.test.python.apache.aurora.config.test_base                                  .....   SUCCESS
src.test.python.apache.aurora.config.test_constraint_parsing                    .....   SUCCESS
src.test.python.apache.aurora.config.test_loader                                .....   SUCCESS
src.test.python.apache.aurora.config.test_thrift                                .....   SUCCESS
src.test.python.apache.aurora.executor.common.directory_sandbox                 .....   SUCCESS
src.test.python.apache.aurora.executor.common.health_checker                    .....   SUCCESS
src.test.python.apache.aurora.executor.common.status_checker                    .....   SUCCESS
src.test.python.apache.aurora.executor.common.task_info                         .....   SUCCESS
src.test.python.apache.aurora.executor.executor_detector                        .....   SUCCESS
src.test.python.apache.aurora.executor.executor_vars                            .....   SUCCESS
src.test.python.apache.aurora.executor.thermos_task_runner                      .....   SUCCESS

Bugs closed: aurora-42

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


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

Branch: refs/heads/master
Commit: b17bc1f96be9bae0bc5b8d7da8735fcb96772f0d
Parents: b6fbcde
Author: Mark Chu-Carroll <mc...@twopensource.com>
Authored: Fri Jan 17 14:51:21 2014 -0500
Committer: Mark Chu-Carroll <mc...@twitter.com>
Committed: Fri Jan 17 14:51:21 2014 -0500

----------------------------------------------------------------------
 .../python/apache/aurora/client/api/__init__.py | 56 ++++++++++----------
 .../apache/aurora/client/api/job_monitor.py     |  2 +-
 .../aurora/client/api/scheduler_client.py       | 18 +++----
 .../apache/aurora/client/commands/core.py       | 10 ++--
 .../aurora/client/api/test_job_monitor.py       |  2 +-
 .../apache/aurora/client/cli/test_create.py     | 18 ++-----
 .../apache/aurora/client/cli/test_diff.py       | 40 +++++++-------
 .../apache/aurora/client/cli/test_kill.py       | 14 ++---
 .../apache/aurora/client/cli/test_status.py     | 26 +++++----
 .../python/apache/aurora/client/cli/util.py     | 50 +++++------------
 .../client/commands/test_cancel_update.py       | 27 ++--------
 .../aurora/client/commands/test_create.py       | 53 ++++++------------
 .../apache/aurora/client/commands/test_diff.py  | 40 +++++++-------
 .../apache/aurora/client/commands/test_kill.py  | 41 ++++----------
 .../aurora/client/commands/test_listjobs.py     | 14 ++---
 .../aurora/client/commands/test_restart.py      | 34 ++++++------
 .../apache/aurora/client/commands/test_run.py   | 10 ++--
 .../apache/aurora/client/commands/test_ssh.py   |  9 ++--
 .../aurora/client/commands/test_status.py       | 20 +++----
 .../aurora/client/commands/test_update.py       | 30 +++++------
 .../apache/aurora/client/commands/util.py       | 21 ++++----
 21 files changed, 226 insertions(+), 309 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/b17bc1f9/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 0fdf1f9..f476f67 100644
--- a/src/main/python/apache/aurora/client/api/__init__.py
+++ b/src/main/python/apache/aurora/client/api/__init__.py
@@ -28,7 +28,7 @@ class AuroraClientAPI(object):
     if not isinstance(cluster, Cluster):
       raise TypeError('AuroraClientAPI expects instance of Cluster for "cluster", got %s' %
           type(cluster))
-    self._scheduler = SchedulerProxy(
+    self._scheduler_proxy = SchedulerProxy(
         cluster, verbose=verbose, session_key_factory=session_key_factory)
     self._cluster = cluster
 
@@ -37,27 +37,27 @@ class AuroraClientAPI(object):
     return self._cluster
 
   @property
-  def scheduler(self):
-    return self._scheduler
+  def scheduler_proxy(self):
+    return self._scheduler_proxy
 
   def create_job(self, config, lock=None):
     log.info('Creating job %s' % config.name())
     log.debug('Full configuration: %s' % config.job())
     log.debug('Lock %s' % lock)
-    return self._scheduler.createJob(config.job(), lock)
+    return self._scheduler_proxy.createJob(config.job(), lock)
 
   def populate_job_config(self, config, validation=None):
-    return self._scheduler.populateJobConfig(config.job(), validation)
+    return self._scheduler_proxy.populateJobConfig(config.job(), validation)
 
   def start_cronjob(self, job_key):
     self._assert_valid_job_key(job_key)
 
     log.info("Starting cron job: %s" % job_key)
-    return self._scheduler.startCronJob(job_key.to_thrift())
+    return self._scheduler_proxy.startCronJob(job_key.to_thrift())
 
   def get_jobs(self, role):
     log.info("Retrieving jobs for role %s" % role)
-    return self._scheduler.getJobs(role)
+    return self._scheduler_proxy.getJobs(role)
 
   def kill_job(self, job_key, instances=None, lock=None):
     log.info("Killing tasks for job: %s" % job_key)
@@ -72,7 +72,7 @@ class AuroraClientAPI(object):
     if instances is not None:
       log.info("Instances to be killed: %s" % instances)
       query.instanceIds = frozenset([int(s) for s in instances])
-    return self._scheduler.killTasks(query, lock)
+    return self._scheduler_proxy.killTasks(query, lock)
 
   def check_status(self, job_key):
     self._assert_valid_job_key(job_key)
@@ -89,7 +89,7 @@ class AuroraClientAPI(object):
                      environment=env)
 
   def query(self, query):
-    return self._scheduler.getTasksStatus(query)
+    return self._scheduler_proxy.getTasksStatus(query)
 
   def update_job(self, config, health_check_interval_seconds=3, instances=None):
     """Run a job update for a given config, for the specified instances.  If
@@ -97,7 +97,7 @@ class AuroraClientAPI(object):
        the update was successful."""
 
     log.info("Updating job: %s" % config.name())
-    updater = Updater(config, health_check_interval_seconds, self._scheduler)
+    updater = Updater(config, health_check_interval_seconds, self._scheduler_proxy)
 
     return updater.update(instances)
 
@@ -107,7 +107,7 @@ class AuroraClientAPI(object):
     self._assert_valid_job_key(job_key)
 
     log.info("Canceling update on job %s" % job_key)
-    resp = Updater.cancel_update(self._scheduler, job_key)
+    resp = Updater.cancel_update(self._scheduler_proxy, job_key)
     if resp.responseCode != ResponseCode.OK:
       log.error('Error cancelling the update: %s' % resp.message)
     return resp
@@ -119,64 +119,64 @@ class AuroraClientAPI(object):
     """
     self._assert_valid_job_key(job_key)
 
-    return Restarter(job_key, updater_config, health_check_interval_seconds, self._scheduler
+    return Restarter(job_key, updater_config, health_check_interval_seconds, self._scheduler_proxy
     ).restart(instances)
 
   def start_maintenance(self, hosts):
     log.info("Starting maintenance for: %s" % hosts.hostNames)
-    return self._scheduler.startMaintenance(hosts)
+    return self._scheduler_proxy.startMaintenance(hosts)
 
   def drain_hosts(self, hosts):
     log.info("Draining tasks on: %s" % hosts.hostNames)
-    return self._scheduler.drainHosts(hosts)
+    return self._scheduler_proxy.drainHosts(hosts)
 
   def maintenance_status(self, hosts):
     log.info("Maintenance status for: %s" % hosts.hostNames)
-    return self._scheduler.maintenanceStatus(hosts)
+    return self._scheduler_proxy.maintenanceStatus(hosts)
 
   def end_maintenance(self, hosts):
     log.info("Ending maintenance for: %s" % hosts.hostNames)
-    return self._scheduler.endMaintenance(hosts)
+    return self._scheduler_proxy.endMaintenance(hosts)
 
   def get_quota(self, role):
     log.info("Getting quota for: %s" % role)
-    return self._scheduler.getQuota(role)
+    return self._scheduler_proxy.getQuota(role)
 
   def set_quota(self, role, cpu, ram_mb, disk_mb):
     log.info("Setting quota for user:%s cpu:%f ram_mb:%d disk_mb: %d"
               % (role, cpu, ram_mb, disk_mb))
-    return self._scheduler.setQuota(role, Quota(cpu, ram_mb, disk_mb))
+    return self._scheduler_proxy.setQuota(role, Quota(cpu, ram_mb, disk_mb))
 
   def force_task_state(self, task_id, status):
     log.info("Requesting that task %s transition to state %s" % (task_id, status))
-    return self._scheduler.forceTaskState(task_id, status)
+    return self._scheduler_proxy.forceTaskState(task_id, status)
 
   def perform_backup(self):
-    return self._scheduler.performBackup()
+    return self._scheduler_proxy.performBackup()
 
   def list_backups(self):
-    return self._scheduler.listBackups()
+    return self._scheduler_proxy.listBackups()
 
   def stage_recovery(self, backup_id):
-    return self._scheduler.stageRecovery(backup_id)
+    return self._scheduler_proxy.stageRecovery(backup_id)
 
   def query_recovery(self, query):
-    return self._scheduler.queryRecovery(query)
+    return self._scheduler_proxy.queryRecovery(query)
 
   def delete_recovery_tasks(self, query):
-    return self._scheduler.deleteRecoveryTasks(query)
+    return self._scheduler_proxy.deleteRecoveryTasks(query)
 
   def commit_recovery(self):
-    return self._scheduler.commitRecovery()
+    return self._scheduler_proxy.commitRecovery()
 
   def unload_recovery(self):
-    return self._scheduler.unloadRecovery()
+    return self._scheduler_proxy.unloadRecovery()
 
   def snapshot(self):
-    return self._scheduler.snapshot()
+    return self._scheduler_proxy.snapshot()
 
   def unsafe_rewrite_config(self, rewrite_request):
-    return self._scheduler.rewriteConfigs(rewrite_request)
+    return self._scheduler_proxy.rewriteConfigs(rewrite_request)
 
   def _assert_valid_job_key(self, job_key):
     if not isinstance(job_key, AuroraJobKey):

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/b17bc1f9/src/main/python/apache/aurora/client/api/job_monitor.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/api/job_monitor.py b/src/main/python/apache/aurora/client/api/job_monitor.py
index f444d1e..3aba215 100644
--- a/src/main/python/apache/aurora/client/api/job_monitor.py
+++ b/src/main/python/apache/aurora/client/api/job_monitor.py
@@ -35,7 +35,7 @@ class JobMonitor(object):
 
   def iter_query(self):
     try:
-      res = self._client.scheduler.getTasksStatus(self._query)
+      res = self._client.scheduler_proxy.getTasksStatus(self._query)
     except TTransport.TTransportException as e:
       print('Failed to query slaves from scheduler: %s' % e)
       return

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/b17bc1f9/src/main/python/apache/aurora/client/api/scheduler_client.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/api/scheduler_client.py b/src/main/python/apache/aurora/client/api/scheduler_client.py
index fb65aee..349634a 100644
--- a/src/main/python/apache/aurora/client/api/scheduler_client.py
+++ b/src/main/python/apache/aurora/client/api/scheduler_client.py
@@ -183,27 +183,27 @@ class SchedulerProxy(object):
     # TODO(Sathya): Make this a part of cluster trait when authentication is pushed to the transport
     # layer.
     self._session_key_factory = session_key_factory
-    self._client = self._scheduler = None
+    self._client = self._scheduler_client = None
     self.verbose = verbose
 
   def with_scheduler(method):
     """Decorator magic to make sure a connection is made to the scheduler"""
     def _wrapper(self, *args, **kwargs):
-      if not self._scheduler:
+      if not self._scheduler_client:
         self._construct_scheduler()
       return method(self, *args, **kwargs)
     return _wrapper
 
   def invalidate(self):
-    self._client = self._scheduler = None
+    self._client = self._scheduler_client = None
 
   @with_scheduler
   def client(self):
     return self._client
 
   @with_scheduler
-  def scheduler(self):
-    return self._scheduler
+  def scheduler_client(self):
+    return self._scheduler_client
 
   def session_key(self):
     try:
@@ -214,15 +214,15 @@ class SchedulerProxy(object):
   def _construct_scheduler(self):
     """
       Populates:
-        self._scheduler
+        self._scheduler_client
         self._client
     """
-    self._scheduler = SchedulerClient.get(self.cluster, verbose=self.verbose)
-    assert self._scheduler, "Could not find scheduler (cluster = %s)" % self.cluster.name
+    self._scheduler_client = SchedulerClient.get(self.cluster, verbose=self.verbose)
+    assert self._scheduler_client, "Could not find scheduler (cluster = %s)" % self.cluster.name
     start = time.time()
     while (time.time() - start) < self.CONNECT_MAXIMUM_WAIT.as_(Time.SECONDS):
       try:
-        self._client = self._scheduler.get_thrift_client()
+        self._client = self._scheduler_client.get_thrift_client()
         break
       except SchedulerClient.CouldNotConnect as e:
         log.warning('Could not connect to scheduler: %s' % e)

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/b17bc1f9/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 edc57b2..b70136b 100644
--- a/src/main/python/apache/aurora/client/commands/core.py
+++ b/src/main/python/apache/aurora/client/commands/core.py
@@ -117,7 +117,7 @@ def create(job_spec, config_file):
   monitor = JobMonitor(api, config.role(), config.environment(), config.name())
   resp = api.create_job(config)
   check_and_log_response(resp)
-  handle_open(api.scheduler.scheduler().url, config.role(), config.environment(), config.name())
+  handle_open(api.scheduler_proxy.scheduler_client().url, config.role(), config.environment(), config.name())
   if options.wait_until == 'RUNNING':
     monitor.wait_until(monitor.running_or_finished)
   elif options.wait_until == 'FINISHED':
@@ -213,7 +213,7 @@ def do_open(args, _):
   api = make_client(cluster_name)
 
   import webbrowser
-  webbrowser.open_new_tab(synthesize_url(api.scheduler.scheduler().url, role, env, job))
+  webbrowser.open_new_tab(synthesize_url(api.scheduler_proxy.scheduler_client().url, role, env, job))
 
 
 @app.command
@@ -298,7 +298,7 @@ def start_cron(args, options):
   config = get_job_config(job_key.to_path(), config_file, options) if config_file else None
   resp = api.start_cronjob(job_key, config=config)
   check_and_log_response(resp)
-  handle_open(api.scheduler.scheduler().url, job_key.role, job_key.env, job_key.name)
+  handle_open(api.scheduler_proxy.scheduler_client().url, job_key.role, job_key.env, job_key.name)
 
 
 @app.command
@@ -369,7 +369,7 @@ def kill(args, options):
   config = get_job_config(job_key.to_path(), config_file, options) if config_file else None
   resp = api.kill_job(job_key, options.shards, config=config)
   check_and_log_response(resp)
-  handle_open(api.scheduler.scheduler().url, job_key.role, job_key.env, job_key.name)
+  handle_open(api.scheduler_proxy.scheduler_client().url, job_key.role, job_key.env, job_key.name)
 
 
 @app.command
@@ -560,7 +560,7 @@ def restart(args, options):
   resp = api.restart(job_key, options.shards, updater_config,
       options.health_check_interval_seconds, config=config)
   check_and_log_response(resp)
-  handle_open(api.scheduler.scheduler().url, job_key.role, job_key.env, job_key.name)
+  handle_open(api.scheduler_proxy.scheduler_client().url, job_key.role, job_key.env, job_key.name)
 
 
 @app.command

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/b17bc1f9/src/test/python/apache/aurora/client/api/test_job_monitor.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/api/test_job_monitor.py b/src/test/python/apache/aurora/client/api/test_job_monitor.py
index 9327b32..491653b 100644
--- a/src/test/python/apache/aurora/client/api/test_job_monitor.py
+++ b/src/test/python/apache/aurora/client/api/test_job_monitor.py
@@ -17,7 +17,7 @@ class JobMonitorTest(MoxTestBase):
     super(JobMonitorTest, self).setUp()
     self.mock_api = self.mox.CreateMock(AuroraClientAPI)
     self.mock_scheduler = self.mox.CreateMock(Client)
-    self.mock_api.scheduler = self.mock_scheduler
+    self.mock_api.scheduler_proxy = self.mock_scheduler
 
   def test_init(self):
     result = Result(scheduleStatusResult=ScheduleStatusResult(tasks=[]))

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/b17bc1f9/src/test/python/apache/aurora/client/cli/test_create.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/test_create.py b/src/test/python/apache/aurora/client/cli/test_create.py
index 2b0c504..deea706 100644
--- a/src/test/python/apache/aurora/client/cli/test_create.py
+++ b/src/test/python/apache/aurora/client/cli/test_create.py
@@ -37,15 +37,6 @@ class TestClientCreateCommand(AuroraClientCommandTest):
     return mock_options
 
   @classmethod
-  def setup_mock_api(cls):
-    """Builds up a mock API object, with a mock SchedulerProxy"""
-    mock_api = Mock(spec=HookedAuroraClientAPI)
-    mock_scheduler = Mock()
-    mock_scheduler.url = "http://something_or_other"
-    mock_api.scheduler = mock_scheduler
-    return (mock_api, mock_scheduler)
-
-  @classmethod
   def create_mock_task(cls, task_id, instance_id, initial_time, status):
     mock_task = Mock(spec=ScheduledTask)
     mock_task.assignedTask = Mock(spec=AssignedTask)
@@ -92,9 +83,8 @@ class TestClientCreateCommand(AuroraClientCommandTest):
 
   @classmethod
   def assert_scheduler_called(cls, mock_api, mock_query, num_queries):
-    print('Calls to getTasksStatus: %s' % mock_api.scheduler.getTasksStatus.call_args_list)
-    assert mock_api.scheduler.getTasksStatus.call_count == num_queries
-    mock_api.scheduler.getTasksStatus.assert_called_with(mock_query)
+    assert mock_api.scheduler_proxy.getTasksStatus.call_count == num_queries
+    mock_api.scheduler_proxy.getTasksStatus.assert_called_with(mock_query)
 
   def test_simple_successful_create_job(self):
     """Run a test of the "create" command against a mocked-out API:
@@ -178,7 +168,7 @@ class TestClientCreateCommand(AuroraClientCommandTest):
       self.assert_create_job_called(api)
 
       # getTasksStatus was called once, before the create_job
-      assert api.scheduler.getTasksStatus.call_count == 1
+      assert api.scheduler_proxy.getTasksStatus.call_count == 1
 
   def test_create_job_failed_invalid_config(self):
     """Run a test of the "create" command against a mocked-out API, with a configuration
@@ -197,4 +187,4 @@ class TestClientCreateCommand(AuroraClientCommandTest):
       # Check that create_job was not called.
       api = mock_context.get_api('west')
       assert api.create_job.call_count == 0
-      assert api.scheduler.getTasksStatus.call_count == 0
+      assert api.scheduler_proxy.getTasksStatus.call_count == 0

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/b17bc1f9/src/test/python/apache/aurora/client/cli/test_diff.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/test_diff.py b/src/test/python/apache/aurora/client/cli/test_diff.py
index b8997e2..726dff3 100644
--- a/src/test/python/apache/aurora/client/cli/test_diff.py
+++ b/src/test/python/apache/aurora/client/cli/test_diff.py
@@ -93,14 +93,14 @@ class TestDiffCommand(AuroraClientCommandTest):
 
   def test_successful_diff(self):
     """Test the diff command."""
-    (mock_api, mock_scheduler) = self.setup_mock_api()
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
     with contextlib.nested(
-        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler),
+        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
         patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS),
         patch('subprocess.call', return_value=0),
         patch('json.loads', return_value=Mock())) as (_, _, subprocess_patch, _):
-      mock_scheduler.getTasksStatus.return_value = self.create_status_response()
-      self.setup_populate_job_config(mock_scheduler)
+      mock_scheduler_proxy.getTasksStatus.return_value = self.create_status_response()
+      self.setup_populate_job_config(mock_scheduler_proxy)
       with temporary_file() as fp:
         fp.write(self.get_valid_config())
         fp.flush()
@@ -108,12 +108,12 @@ class TestDiffCommand(AuroraClientCommandTest):
         cmd.execute(['job', 'diff', 'west/bozo/test/hello', fp.name])
 
         # Diff should get the task status, populate a config, and run diff.
-        mock_scheduler.getTasksStatus.assert_called_with(
+        mock_scheduler_proxy.getTasksStatus.assert_called_with(
             TaskQuery(jobName='hello', environment='test', owner=Identity(role='bozo'),
                 statuses=ACTIVE_STATES))
-        assert mock_scheduler.populateJobConfig.call_count == 1
-        assert isinstance(mock_scheduler.populateJobConfig.call_args[0][0], JobConfiguration)
-        assert (mock_scheduler.populateJobConfig.call_args[0][0].key ==
+        assert mock_scheduler_proxy.populateJobConfig.call_count == 1
+        assert isinstance(mock_scheduler_proxy.populateJobConfig.call_args[0][0], JobConfiguration)
+        assert (mock_scheduler_proxy.populateJobConfig.call_args[0][0].key ==
             JobKey(environment=u'test', role=u'bozo', name=u'hello'))
         # Subprocess should have been used to invoke diff with two parameters.
         assert subprocess_patch.call_count == 1
@@ -123,11 +123,11 @@ class TestDiffCommand(AuroraClientCommandTest):
   def test_diff_invalid_config(self):
     """Test the diff command if the user passes a config with an error in it."""
     mock_options = self.setup_mock_options()
-    (mock_api, mock_scheduler) = self.create_mock_api()
-    mock_scheduler.getTasksStatus.return_value = self.create_status_response()
-    self.setup_populate_job_config(mock_scheduler)
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
+    mock_scheduler_proxy.getTasksStatus.return_value = self.create_status_response()
+    self.setup_populate_job_config(mock_scheduler_proxy)
     with contextlib.nested(
-        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler),
+        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
         patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS),
         patch('twitter.common.app.get_options', return_value=mock_options),
         patch('subprocess.call', return_value=0),
@@ -143,18 +143,18 @@ class TestDiffCommand(AuroraClientCommandTest):
         cmd = AuroraCommandLine()
         result = cmd.execute(['job', 'diff', 'west/bozo/test/hello', fp.name])
         assert result == EXIT_INVALID_CONFIGURATION
-        assert mock_scheduler.getTasksStatus.call_count == 0
-        assert mock_scheduler.populateJobConfig.call_count == 0
+        assert mock_scheduler_proxy.getTasksStatus.call_count == 0
+        assert mock_scheduler_proxy.populateJobConfig.call_count == 0
         assert subprocess_patch.call_count == 0
 
   def test_diff_server_error(self):
     """Test the diff command if the user passes a config with an error in it."""
     mock_options = self.setup_mock_options()
-    (mock_api, mock_scheduler) = self.create_mock_api()
-    mock_scheduler.getTasksStatus.return_value = self.create_failed_status_response()
-    self.setup_populate_job_config(mock_scheduler)
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
+    mock_scheduler_proxy.getTasksStatus.return_value = self.create_failed_status_response()
+    self.setup_populate_job_config(mock_scheduler_proxy)
     with contextlib.nested(
-        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler),
+        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
         patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS),
         patch('twitter.common.app.get_options', return_value=mock_options),
         patch('subprocess.call', return_value=0),
@@ -172,8 +172,8 @@ class TestDiffCommand(AuroraClientCommandTest):
         assert result == EXIT_INVALID_PARAMETER
         # In this error case, we should have called the server getTasksStatus;
         # but since it fails, we shouldn't call populateJobConfig or subprocess.
-        mock_scheduler.getTasksStatus.assert_called_with(
+        mock_scheduler_proxy.getTasksStatus.assert_called_with(
             TaskQuery(jobName='hello', environment='test', owner=Identity(role='bozo'),
                 statuses=ACTIVE_STATES))
-        assert mock_scheduler.populateJobConfig.call_count == 0
+        assert mock_scheduler_proxy.populateJobConfig.call_count == 0
         assert subprocess_patch.call_count == 0

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/b17bc1f9/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 66d729d..0ee8b85 100644
--- a/src/test/python/apache/aurora/client/cli/test_kill.py
+++ b/src/test/python/apache/aurora/client/cli/test_kill.py
@@ -39,14 +39,14 @@ class TestClientKillCommand(AuroraClientCommandTest):
   def test_kill_job(self):
     """Test kill client-side API logic."""
     mock_context = FakeAuroraCommandContext()
-    mock_scheduler = Mock()
+    mock_scheduler_proxy = Mock()
     with contextlib.nested(
         patch('apache.aurora.client.cli.jobs.Job.create_context', return_value=mock_context),
         patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS)):
 
       api = mock_context.get_api('west')
       api.kill_job.return_value = self.get_kill_job_response()
-      mock_scheduler.scheduler.killTasks.return_value = self.get_kill_job_response()
+      mock_scheduler_proxy.killTasks.return_value = self.get_kill_job_response()
       with temporary_file() as fp:
         fp.write(self.get_valid_config())
         fp.flush()
@@ -79,11 +79,11 @@ class TestClientKillCommand(AuroraClientCommandTest):
 
   def test_kill_job_with_instances_deep_api(self):
     """Test kill client-side API logic."""
-    (mock_api, mock_scheduler) = self.setup_mock_api()
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
     with contextlib.nested(
-        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler),
+        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
         patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS)):
-      mock_scheduler.killTasks.return_value = self.get_kill_job_response()
+      mock_scheduler_proxy.killTasks.return_value = self.get_kill_job_response()
       with temporary_file() as fp:
         fp.write(self.get_valid_config())
         fp.flush()
@@ -91,7 +91,7 @@ class TestClientKillCommand(AuroraClientCommandTest):
         cmd.execute(['job', 'kill', '--config=%s' % fp.name, '--instances=0,2,4-6',
            'west/bozo/test/hello'])
       # Now check that the right API calls got made.
-      assert mock_scheduler.killTasks.call_count == 1
-      mock_scheduler.killTasks.assert_called_with(
+      assert mock_scheduler_proxy.killTasks.call_count == 1
+      mock_scheduler_proxy.killTasks.assert_called_with(
         TaskQuery(jobName='hello', environment='test', instanceIds=frozenset([0, 2, 4, 5, 6]),
             owner=Identity(role='bozo')), None)

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/b17bc1f9/src/test/python/apache/aurora/client/cli/test_status.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/test_status.py b/src/test/python/apache/aurora/client/cli/test_status.py
index 05f3929..45c3edf 100644
--- a/src/test/python/apache/aurora/client/cli/test_status.py
+++ b/src/test/python/apache/aurora/client/cli/test_status.py
@@ -97,14 +97,14 @@ class TestJobStatus(AuroraClientCommandTest):
   def test_successful_status_deep(self):
     """Test the status command more deeply: in a request with a fully specified
     job, it should end up doing a query using getTasksStatus."""
-    (mock_api, mock_scheduler) = self.setup_mock_api()
-    mock_scheduler.query.return_value = self.create_status_response()
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
+    mock_scheduler_proxy.query.return_value = self.create_status_response()
     with contextlib.nested(
-        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler),
+        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
         patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS)):
       cmd = AuroraCommandLine()
       cmd.execute(['job', 'status', 'west/bozo/test/hello'])
-      mock_scheduler.getTasksStatus.assert_called_with(TaskQuery(jobName='hello',
+      mock_scheduler_proxy.getTasksStatus.assert_called_with(TaskQuery(jobName='hello',
           environment='test', owner=Identity(role='bozo')))
 
   def test_status_wildcard(self):
@@ -123,10 +123,17 @@ class TestJobStatus(AuroraClientCommandTest):
     # Wildcard should have expanded to two jobs, so there should be two calls
     # to check_status.
     assert mock_api.check_status.call_count == 2
-    assert (call(AuroraJobKey('west', 'RoleA', 'test', 'hithere')) in
-        mock_api.check_status.call_args_list)
-    assert (call(AuroraJobKey('west', 'bozo', 'test', 'hello')) in
-        mock_api.check_status.call_args_list)
+
+    assert mock_api.check_status.call_args_list[0][0][0].cluster == 'west'
+    assert mock_api.check_status.call_args_list[0][0][0].role == 'RoleA'
+    assert mock_api.check_status.call_args_list[0][0][0].env == 'test'
+    assert mock_api.check_status.call_args_list[0][0][0].name == 'hithere'
+
+    assert mock_api.check_status.call_args_list[1][0][0].cluster == 'west'
+    assert mock_api.check_status.call_args_list[1][0][0].role == 'bozo'
+    assert mock_api.check_status.call_args_list[1][0][0].env == 'test'
+    assert mock_api.check_status.call_args_list[1][0][0].name == 'hello'
+
 
   def test_status_wildcard_two(self):
     """Test status using a wildcard. It should first call api.get_jobs, and then do a
@@ -149,11 +156,10 @@ class TestJobStatus(AuroraClientCommandTest):
   def test_unsuccessful_status_shallow(self):
     """Test the status command at the shallowest level: calling status should end up invoking
     the local APIs get_status method."""
-    # Calls api.check_status, which calls scheduler.getJobs
+    # Calls api.check_status, which calls scheduler_proxy.getJobs
     mock_context = FakeAuroraCommandContext()
     mock_api = mock_context.get_api('west')
     mock_api.check_status.return_value = self.create_failed_status_response()
-    #    mock_api.scheduler.getTasksStatus.return_value =
     with contextlib.nested(
         patch('apache.aurora.client.cli.jobs.Job.create_context', return_value=mock_context)):
       cmd = AuroraCommandLine()

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/b17bc1f9/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 2985865..79a497b 100644
--- a/src/test/python/apache/aurora/client/cli/util.py
+++ b/src/test/python/apache/aurora/client/cli/util.py
@@ -19,7 +19,7 @@ class FakeAuroraCommandContext(AuroraCommandContext):
     super(FakeAuroraCommandContext, self).__init__()
     self.options = None
     self.status = []
-    self.fake_api = self.setup_fake_api()
+    self.fake_api = self.create_mock_api()
     self.task_status = []
     self.showed_urls = []
 
@@ -27,29 +27,21 @@ class FakeAuroraCommandContext(AuroraCommandContext):
     return self.fake_api
 
   @classmethod
-  def setup_mock_api(cls):
+  def create_mock_api(cls):
     """Builds up a mock API object, with a mock SchedulerProxy.
     Returns the API and the proxy"""
-
-    mock_scheduler = Mock()
-    mock_scheduler.url = "http://something_or_other"
-    mock_scheduler_client = Mock()
-    mock_scheduler_client.scheduler.return_value = mock_scheduler
-    mock_scheduler_client.url = "http://something_or_other"
+    # This looks strange, but we set up the same object to use as both
+    # the SchedulerProxy and the SchedulerClient. These tests want to observe
+    # what API calls get made against the scheduler, and both of these objects
+    # delegate calls to the scheduler. It doesn't matter which one is used:
+    # what we care about is that the right API calls get made.
     mock_api = Mock(spec=HookedAuroraClientAPI)
-    mock_api.scheduler = mock_scheduler_client
-    return (mock_api, mock_scheduler_client)
-
-  def setup_fake_api(self):
-    # In here, we'd like to get it mocked so that the HookedAuroraClientAPI
-    # object, and its underlying AuroraClientAPI objects are not
-    # mocked, but the scheduler object is.
-    new_fake = Mock(spec=HookedAuroraClientAPI)
-    new_fake.scheduler = Mock()
-    new_fake.scheduler.url = 'http://something_or_other'
-#    new_fake.scheduler.getTasksStatus.side_effect = []
-    self.fake_api = new_fake
-    return self.fake_api
+    mock_scheduler_proxy = Mock()
+    mock_scheduler_proxy.url = "http://something_or_other"
+    mock_scheduler_proxy.scheduler_client.return_value = mock_scheduler_proxy
+    mock_api = Mock(spec=HookedAuroraClientAPI)
+    mock_api.scheduler_proxy = mock_scheduler_proxy
+    return mock_api
 
   def open_page(self, url):
     self.showed_urls.append(url)
@@ -60,7 +52,7 @@ class FakeAuroraCommandContext(AuroraCommandContext):
   def add_expected_status_query_result(self, expected_result):
     self.task_status.append(expected_result)
     # each call adds an expected query result, in order.
-    self.fake_api.scheduler.getTasksStatus.side_effect = self.task_status
+    self.fake_api.scheduler_proxy.getTasksStatus.side_effect = self.task_status
 
 
 class AuroraClientCommandTest(unittest.TestCase):
@@ -72,20 +64,6 @@ class AuroraClientCommandTest(unittest.TestCase):
     response.result = Mock(spec=Result)
     return response
 
-  @classmethod
-  def setup_mock_api(cls):
-    """Builds up a mock API object, with a mock SchedulerProxy.
-    Returns the API and the proxy"""
-    # TODO: merge this with setup_fake_api (MESOS-4861)
-    mock_scheduler = Mock()
-    mock_scheduler.url = "http://something_or_other"
-    mock_scheduler_client = Mock()
-    mock_scheduler_client.scheduler.return_value = mock_scheduler
-    mock_scheduler_client.url = "http://something_or_other"
-    mock_api = Mock(spec=HookedAuroraClientAPI)
-    mock_api.scheduler = mock_scheduler_client
-    return (mock_api, mock_scheduler_client)
-
 
   @classmethod
   def create_simple_success_response(cls):

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/b17bc1f9/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 b1d63cc..4765f72 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
@@ -69,9 +69,6 @@ class TestClientCancelUpdateCommand(AuroraClientCommandTest):
         AuroraJobKey(cls.TEST_CLUSTER, cls.TEST_ROLE, cls.TEST_ENV, cls.TEST_JOB),
         config=None)
 
-  @classmethod
-  def assert_scheduler_called(cls, mock_api):
-    assert mock_api.scheduler.scheduler.call_count == 1
 
   def test_simple_successful_cancel_update(self):
     """Run a test of the "kill" command against a mocked-out API:
@@ -92,20 +89,6 @@ class TestClientCancelUpdateCommand(AuroraClientCommandTest):
       self.assert_cancel_update_called(mock_api)
 
   @classmethod
-  def setup_mock_api(cls):
-    """Builds up a mock API object, with a mock SchedulerProxy.
-    Returns the API and the proxy"""
-
-    mock_scheduler = Mock()
-    mock_scheduler.url = "http://something_or_other"
-    mock_scheduler_client = Mock()
-    mock_scheduler_client.scheduler.return_value = mock_scheduler
-    mock_scheduler_client.url = "http://something_or_other"
-    mock_api = Mock(spec=HookedAuroraClientAPI)
-    mock_api.scheduler = mock_scheduler_client
-    return (mock_api, mock_scheduler_client)
-
-  @classmethod
   def get_expected_task_query(cls, shards=None):
     instance_ids = frozenset(shards) if shards is not None else None
     # Helper to create the query that will be a parameter to job kill.
@@ -124,10 +107,10 @@ class TestClientCancelUpdateCommand(AuroraClientCommandTest):
     mock_config = Mock()
     mock_config.hooks = []
     mock_config.raw.return_value.enable_hooks.return_value.get.return_value = False
-    (mock_api, mock_scheduler) = self.setup_mock_api()
-    mock_scheduler.releaseLock.return_value = self.get_release_lock_response()
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
+    mock_scheduler_proxy.releaseLock.return_value = self.get_release_lock_response()
     with contextlib.nested(
-        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler),
+        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
         patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS),
         patch('twitter.common.app.get_options', return_value=mock_options),
         patch('apache.aurora.client.commands.core.get_job_config', return_value=mock_config)) as (
@@ -139,6 +122,6 @@ class TestClientCancelUpdateCommand(AuroraClientCommandTest):
 
       # All that cancel_update really does is release the update lock.
       # So that's all we really need to check.
-      assert mock_scheduler.releaseLock.call_count == 1
-      assert mock_scheduler.releaseLock.call_args[0][0].key.job == JobKey(environment='test',
+      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')

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/b17bc1f9/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 57a6dc8..557e2ac 100644
--- a/src/test/python/apache/aurora/client/commands/test_create.py
+++ b/src/test/python/apache/aurora/client/commands/test_create.py
@@ -39,15 +39,6 @@ class TestClientCreateCommand(AuroraClientCommandTest):
     return mock_options
 
   @classmethod
-  def setup_mock_api(cls):
-    """Builds up a mock API object, with a mock SchedulerProxy"""
-    mock_api = Mock(spec=HookedAuroraClientAPI)
-    mock_scheduler = Mock()
-    mock_scheduler.url = "http://something_or_other"
-    mock_api.scheduler = mock_scheduler
-    return (mock_api, mock_scheduler)
-
-  @classmethod
   def create_mock_task(cls, task_id, instance_id, initial_time, status):
     mock_task = Mock(spec=ScheduledTask)
     mock_task.assignedTask = Mock(spec=AssignedTask)
@@ -95,9 +86,8 @@ class TestClientCreateCommand(AuroraClientCommandTest):
   @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.scheduler.call_count == 1
-    assert mock_api.scheduler.getTasksStatus.call_count == num_queries
-    mock_api.scheduler.getTasksStatus.assert_called_with(mock_query)
+    assert mock_api.scheduler_proxy.getTasksStatus.call_count == num_queries
+    mock_api.scheduler_proxy.getTasksStatus.assert_called_with(mock_query)
 
   def test_simple_successful_create_job(self):
     """Run a test of the "create" command against a mocked-out API:
@@ -109,7 +99,7 @@ class TestClientCreateCommand(AuroraClientCommandTest):
     # 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) = self.setup_mock_api()
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
     with contextlib.nested(
         patch('apache.aurora.client.commands.core.make_client', return_value=mock_api),
         patch('twitter.common.app.get_options', return_value=mock_options)) as (make_client,
@@ -119,7 +109,7 @@ class TestClientCreateCommand(AuroraClientCommandTest):
       # The monitor uses TaskQuery to get the tasks. It's called at least twice:once before
       # the job is created, and once after. So we need to set up mocks for the query results.
       mock_query = self.create_mock_query()
-      mock_scheduler.getTasksStatus.side_effect = [
+      mock_scheduler_proxy.getTasksStatus.side_effect = [
         self.create_mock_status_query_result(ScheduleStatus.INIT),
         self.create_mock_status_query_result(ScheduleStatus.RUNNING)
       ]
@@ -127,9 +117,6 @@ class TestClientCreateCommand(AuroraClientCommandTest):
       # With the monitor set up, create finally gets around to calling create_job.
       mock_api.create_job.return_value = self.get_createjob_response()
 
-      # Then it calls handle_open; we need to provide a mock for the API calls it uses.
-      mock_api.scheduler.scheduler.return_value = mock_scheduler
-
       # 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.
 
@@ -151,7 +138,7 @@ class TestClientCreateCommand(AuroraClientCommandTest):
     this time, make the monitor check status several times before successful completion.
     """
     mock_options = self.setup_mock_options()
-    (mock_api, mock_scheduler) = self.setup_mock_api()
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
     with contextlib.nested(
         patch('time.sleep'),
         patch('apache.aurora.client.commands.core.make_client', return_value=mock_api),
@@ -165,9 +152,8 @@ class TestClientCreateCommand(AuroraClientCommandTest):
           self.create_mock_status_query_result(ScheduleStatus.RUNNING),
           self.create_mock_status_query_result(ScheduleStatus.FINISHED)
       ]
-      mock_scheduler.getTasksStatus.side_effect = mock_query_results
+      mock_scheduler_proxy.getTasksStatus.side_effect = mock_query_results
       mock_api.create_job.return_value = self.get_createjob_response()
-      mock_api.scheduler.scheduler.return_value = mock_scheduler
       with temporary_file() as fp:
         fp.write(self.get_valid_config())
         fp.flush()
@@ -185,7 +171,7 @@ class TestClientCreateCommand(AuroraClientCommandTest):
     this time, make the monitor check status several times before successful completion.
     """
     mock_options = self.setup_mock_options()
-    (mock_api, mock_scheduler) = self.setup_mock_api()
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
     with contextlib.nested(
         patch('apache.aurora.client.commands.core.make_client', return_value=mock_api),
         patch('twitter.common.app.get_options', return_value=mock_options)) as (make_client,
@@ -194,7 +180,7 @@ class TestClientCreateCommand(AuroraClientCommandTest):
       mock_query_results = [
           self.create_mock_status_query_result(ScheduleStatus.INIT)
       ]
-      mock_scheduler.getTasksStatus.side_effect = mock_query_results
+      mock_scheduler_proxy.getTasksStatus.side_effect = mock_query_results
       mock_api.create_job.return_value = self.get_failed_createjob_response()
       # This is the real test: invoke create as if it had been called by the command line.
       with temporary_file() as fp:
@@ -206,11 +192,9 @@ class TestClientCreateCommand(AuroraClientCommandTest):
       # Check that create_job was called exactly once, with an AuroraConfig parameter.
       self.assert_create_job_called(mock_api)
 
-      # scheduler.scheduler() should not have been called, because the create_job failed.
-      assert mock_api.scheduler.scheduler.call_count == 0
       # getTasksStatus was called once, before the create_job
-      assert mock_scheduler.getTasksStatus.call_count == 1
-      mock_scheduler.getTasksStatus.assert_called_with(mock_query)
+      assert mock_scheduler_proxy.getTasksStatus.call_count == 1
+      mock_scheduler_proxy.getTasksStatus.assert_called_with(mock_query)
       # make_client should have been called once.
       make_client.assert_called_with('west')
 
@@ -219,7 +203,7 @@ class TestClientCreateCommand(AuroraClientCommandTest):
     this time, make the monitor check status several times before successful completion.
     """
     mock_options = self.setup_mock_options()
-    (mock_api, mock_scheduler) = self.setup_mock_api()
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
     with contextlib.nested(
         patch('time.sleep'),
         patch('apache.aurora.client.commands.core.make_client', return_value=mock_api),
@@ -233,9 +217,8 @@ class TestClientCreateCommand(AuroraClientCommandTest):
           self.create_mock_status_query_result(ScheduleStatus.RUNNING),
           self.create_mock_status_query_result(ScheduleStatus.FINISHED)
       ]
-      mock_scheduler.getTasksStatus.side_effect = mock_query_results
+      mock_scheduler_proxy.getTasksStatus.side_effect = mock_query_results
       mock_api.create_job.return_value = self.get_createjob_response()
-      mock_api.scheduler.scheduler.return_value = mock_scheduler
       with temporary_file() as fp:
         fp.write(self.get_valid_config())
         fp.flush()
@@ -251,7 +234,7 @@ class TestClientCreateCommand(AuroraClientCommandTest):
     """Run a test of the "create" command against a mocked-out API, with a configuration
     containing a syntax error"""
     mock_options = self.setup_mock_options()
-    (mock_api, mock_scheduler) = self.setup_mock_api()
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
     with contextlib.nested(
         patch('apache.aurora.client.commands.core.make_client', return_value=mock_api),
         patch('twitter.common.app.get_options', return_value=mock_options)) as (make_client,
@@ -265,10 +248,8 @@ class TestClientCreateCommand(AuroraClientCommandTest):
       # Now check that the right API calls got made.
       # Check that create_job was not called.
       assert mock_api.create_job.call_count == 0
-      # scheduler.scheduler() should not have been called, because the config was invalid.
-      assert mock_api.scheduler.scheduler.call_count == 0
 
-      assert mock_scheduler.getTasksStatus.call_count == 0
+      assert mock_scheduler_proxy.getTasksStatus.call_count == 0
       # make_client should not have been called.
       assert make_client.call_count == 0
 
@@ -277,7 +258,7 @@ class TestClientCreateCommand(AuroraClientCommandTest):
     this time, make the monitor check status several times before successful completion.
     """
     mock_options = self.setup_mock_options()
-    (mock_api, mock_scheduler) = self.setup_mock_api()
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
     with contextlib.nested(
         patch('apache.aurora.client.commands.core.make_client', return_value=mock_api),
         patch('twitter.common.app.get_options', return_value=mock_options)) as (make_client,
@@ -290,9 +271,7 @@ class TestClientCreateCommand(AuroraClientCommandTest):
       # Now check that the right API calls got made.
       # Check that create_job was not called.
       assert mock_api.create_job.call_count == 0
-      # scheduler.scheduler() should not have been called, because the config was invalid.
-      assert mock_api.scheduler.scheduler.call_count == 0
       # getTasksStatus was called once, before the create_job
-      assert mock_scheduler.getTasksStatus.call_count == 0
+      assert mock_scheduler_proxy.getTasksStatus.call_count == 0
       # make_client should not have been called.
       assert make_client.call_count == 0

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/b17bc1f9/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 a042153..3f07805 100644
--- a/src/test/python/apache/aurora/client/commands/test_diff.py
+++ b/src/test/python/apache/aurora/client/commands/test_diff.py
@@ -91,11 +91,11 @@ class TestDiffCommand(AuroraClientCommandTest):
   def test_successful_diff(self):
     """Test the diff command."""
     mock_options = self.setup_mock_options()
-    (mock_api, mock_scheduler) = self.create_mock_api()
-    mock_scheduler.getTasksStatus.return_value = self.create_status_response()
-    self.setup_populate_job_config(mock_scheduler)
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
+    mock_scheduler_proxy.getTasksStatus.return_value = self.create_status_response()
+    self.setup_populate_job_config(mock_scheduler_proxy)
     with contextlib.nested(
-        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler),
+        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
         patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS),
         patch('twitter.common.app.get_options', return_value=mock_options),
         patch('subprocess.call', return_value=0),
@@ -111,12 +111,12 @@ class TestDiffCommand(AuroraClientCommandTest):
         diff(['west/mchucarroll/test/hello', fp.name])
 
         # Diff should get the task status, populate a config, and run diff.
-        mock_scheduler.getTasksStatus.assert_called_with(
+        mock_scheduler_proxy.getTasksStatus.assert_called_with(
             TaskQuery(jobName='hello', environment='test', owner=Identity(role='mchucarroll'),
                 statuses=ACTIVE_STATES))
-        assert mock_scheduler.populateJobConfig.call_count == 1
-        assert isinstance(mock_scheduler.populateJobConfig.call_args[0][0], JobConfiguration)
-        assert (mock_scheduler.populateJobConfig.call_args[0][0].key ==
+        assert mock_scheduler_proxy.populateJobConfig.call_count == 1
+        assert isinstance(mock_scheduler_proxy.populateJobConfig.call_args[0][0], JobConfiguration)
+        assert (mock_scheduler_proxy.populateJobConfig.call_args[0][0].key ==
             JobKey(environment=u'test', role=u'mchucarroll', name=u'hello'))
         # Subprocess should have been used to invoke diff with two parameters.
         assert subprocess_patch.call_count == 1
@@ -126,11 +126,11 @@ class TestDiffCommand(AuroraClientCommandTest):
   def test_diff_invalid_config(self):
     """Test the diff command if the user passes a config with an error in it."""
     mock_options = self.setup_mock_options()
-    (mock_api, mock_scheduler) = self.create_mock_api()
-    mock_scheduler.getTasksStatus.return_value = self.create_status_response()
-    self.setup_populate_job_config(mock_scheduler)
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
+    mock_scheduler_proxy.getTasksStatus.return_value = self.create_status_response()
+    self.setup_populate_job_config(mock_scheduler_proxy)
     with contextlib.nested(
-        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler),
+        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
         patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS),
         patch('twitter.common.app.get_options', return_value=mock_options),
         patch('subprocess.call', return_value=0),
@@ -145,18 +145,18 @@ class TestDiffCommand(AuroraClientCommandTest):
         fp.flush()
         self.assertRaises(Config.InvalidConfigError, diff,
             ['west/mchucarroll/test/hello', fp.name])
-        assert mock_scheduler.getTasksStatus.call_count == 0
-        assert mock_scheduler.populateJobConfig.call_count == 0
+        assert mock_scheduler_proxy.getTasksStatus.call_count == 0
+        assert mock_scheduler_proxy.populateJobConfig.call_count == 0
         assert subprocess_patch.call_count == 0
 
   def test_diff_server_error(self):
     """Test the diff command if the user passes a config with an error in it."""
     mock_options = self.setup_mock_options()
-    (mock_api, mock_scheduler) = self.create_mock_api()
-    mock_scheduler.getTasksStatus.return_value = self.create_failed_status_response()
-    self.setup_populate_job_config(mock_scheduler)
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
+    mock_scheduler_proxy.getTasksStatus.return_value = self.create_failed_status_response()
+    self.setup_populate_job_config(mock_scheduler_proxy)
     with contextlib.nested(
-        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler),
+        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
         patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS),
         patch('twitter.common.app.get_options', return_value=mock_options),
         patch('subprocess.call', return_value=0),
@@ -173,8 +173,8 @@ class TestDiffCommand(AuroraClientCommandTest):
             ['west/mchucarroll/test/hello', fp.name])
         # In this error case, we should have called the server getTasksStatus;
         # but since it fails, we shouldn't call populateJobConfig or subprocess.
-        mock_scheduler.getTasksStatus.assert_called_with(
+        mock_scheduler_proxy.getTasksStatus.assert_called_with(
             TaskQuery(jobName='hello', environment='test', owner=Identity(role='mchucarroll'),
                 statuses=ACTIVE_STATES))
-        assert mock_scheduler.populateJobConfig.call_count == 0
+        assert mock_scheduler_proxy.populateJobConfig.call_count == 0
         assert subprocess_patch.call_count == 0

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/b17bc1f9/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 edca5dc..39976d0 100644
--- a/src/test/python/apache/aurora/client/commands/test_kill.py
+++ b/src/test/python/apache/aurora/client/commands/test_kill.py
@@ -64,10 +64,6 @@ class TestClientKillCommand(AuroraClientCommandTest):
   def assert_kill_job_called(cls, mock_api):
     assert mock_api.kill_job.call_count == 1
 
-  @classmethod
-  def assert_scheduler_called(cls, mock_api):
-    # scheduler.scheduler() is called once, as a part of the handle_open call.
-    assert mock_api.scheduler.scheduler.call_count == 1
 
   def test_simple_successful_kill_job(self):
     """Run a test of the "kill" command against a mocked-out API:
@@ -95,24 +91,9 @@ class TestClientKillCommand(AuroraClientCommandTest):
       mock_api.kill_job.assert_called_with(
         AuroraJobKey(cluster=self.TEST_CLUSTER, role=self.TEST_ROLE, env=self.TEST_ENV,
             name=self.TEST_JOB), None, config=mock_config)
-      self.assert_scheduler_called(mock_api)
       assert mock_make_client_factory.call_count == 1
 
   @classmethod
-  def setup_mock_api(cls):
-    """Builds up a mock API object, with a mock SchedulerProxy.
-    Returns the API and the proxy"""
-
-    mock_scheduler = Mock()
-    mock_scheduler.url = "http://something_or_other"
-    mock_scheduler_client = Mock()
-    mock_scheduler_client.scheduler.return_value = mock_scheduler
-    mock_scheduler_client.url = "http://something_or_other"
-    mock_api = Mock(spec=HookedAuroraClientAPI)
-    mock_api.scheduler = mock_scheduler_client
-    return (mock_api, mock_scheduler_client)
-
-  @classmethod
   def get_expected_task_query(cls, shards=None):
     """Helper to create the query that will be a parameter to job kill."""
     instance_ids = frozenset(shards) if shards is not None else None
@@ -125,12 +106,12 @@ class TestClientKillCommand(AuroraClientCommandTest):
     mock_config = Mock()
     mock_config.hooks = []
     mock_config.raw.return_value.enable_hooks.return_value.get.return_value = False
-    (mock_api, mock_scheduler) = self.setup_mock_api()
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
     mock_api_factory = Mock(return_value=mock_api)
-    mock_scheduler.killTasks.return_value = self.get_kill_job_response()
+    mock_scheduler_proxy.killTasks.return_value = self.get_kill_job_response()
     with contextlib.nested(
         patch('apache.aurora.client.factory.make_client_factory', return_value=mock_api_factory),
-        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler),
+        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
         patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS),
         patch('twitter.common.app.get_options', return_value=mock_options),
         patch('apache.aurora.client.commands.core.get_job_config', return_value=mock_config)) as (
@@ -144,9 +125,8 @@ class TestClientKillCommand(AuroraClientCommandTest):
         kill(['west/mchucarroll/test/hello', fp.name], mock_options)
 
       # Now check that the right API calls got made.
-      self.assert_scheduler_called(mock_api)
-      assert mock_scheduler.killTasks.call_count == 1
-      mock_scheduler.killTasks.assert_called_with(self.get_expected_task_query(), None)
+      assert mock_scheduler_proxy.killTasks.call_count == 1
+      mock_scheduler_proxy.killTasks.assert_called_with(self.get_expected_task_query(), None)
 
   def test_kill_job_api_level_with_shards(self):
     """Test kill client-side API logic."""
@@ -155,12 +135,12 @@ class TestClientKillCommand(AuroraClientCommandTest):
     mock_config = Mock()
     mock_config.hooks = []
     mock_config.raw.return_value.enable_hooks.return_value.get.return_value = False
-    (mock_api, mock_scheduler) = self.setup_mock_api()
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
     mock_api_factory = Mock(return_value=mock_api)
-    mock_scheduler.killTasks.return_value = self.get_kill_job_response()
+    mock_scheduler_proxy.killTasks.return_value = self.get_kill_job_response()
     with contextlib.nested(
         patch('apache.aurora.client.factory.make_client_factory', return_value=mock_api_factory),
-        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler),
+        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
         patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS),
         patch('twitter.common.app.get_options', return_value=mock_options),
         patch('apache.aurora.client.commands.core.get_job_config', return_value=mock_config)) as (
@@ -174,7 +154,6 @@ class TestClientKillCommand(AuroraClientCommandTest):
         kill(['west/mchucarroll/test/hello', fp.name], mock_options)
 
       # Now check that the right API calls got made.
-      self.assert_scheduler_called(mock_api)
-      assert mock_scheduler.killTasks.call_count == 1
+      assert mock_scheduler_proxy.killTasks.call_count == 1
       query = self.get_expected_task_query([0, 1, 2, 3])
-      mock_scheduler.killTasks.assert_called_with(query, None)
+      mock_scheduler_proxy.killTasks.assert_called_with(query, None)

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/b17bc1f9/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 16973d4..1ea6a52 100644
--- a/src/test/python/apache/aurora/client/commands/test_listjobs.py
+++ b/src/test/python/apache/aurora/client/commands/test_listjobs.py
@@ -43,10 +43,10 @@ class TestListJobs(AuroraClientCommandTest):
   def test_successful_listjobs(self):
     """Test the list_jobs command."""
     mock_options = self.setup_mock_options()
-    (mock_api, mock_scheduler) = self.create_mock_api()
-    mock_scheduler.getJobs.return_value = self.create_listjobs_response()
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
+    mock_scheduler_proxy.getJobs.return_value = self.create_listjobs_response()
     with contextlib.nested(
-        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler),
+        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
         patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS),
         patch('twitter.common.app.get_options', return_value=mock_options)) as (
             mock_scheduler_proxy_class,
@@ -54,15 +54,15 @@ class TestListJobs(AuroraClientCommandTest):
             options):
       list_jobs(['west/mchucarroll'])
 
-      mock_scheduler.getJobs.assert_called_with(self.TEST_ROLE)
+      mock_scheduler_proxy.getJobs.assert_called_with(self.TEST_ROLE)
 
   def test_listjobs_badcluster(self):
     """Test the list_jobs command when the user provides an invalid cluster."""
     mock_options = self.setup_mock_options()
-    (mock_api, mock_scheduler) = self.create_mock_api()
-    mock_scheduler.getJobs.return_value = self.create_listjobs_response()
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
+    mock_scheduler_proxy.getJobs.return_value = self.create_listjobs_response()
     with contextlib.nested(
-        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler),
+        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
         patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS),
         patch('twitter.common.app.get_options', return_value=mock_options)) as (
             mock_scheduler_proxy_class,

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/b17bc1f9/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 77a24e8..f3040e4 100644
--- a/src/test/python/apache/aurora/client/commands/test_restart.py
+++ b/src/test/python/apache/aurora/client/commands/test_restart.py
@@ -40,7 +40,7 @@ class TestRestartCommand(AuroraClientCommandTest):
   @classmethod
   def setup_mock_scheduler_for_simple_restart(cls, api):
     """Set up all of the API mocks for scheduler calls during a simple restart"""
-    sched_proxy = api.scheduler
+    sched_proxy = api.scheduler_proxy
     cls.setup_get_tasks_status_calls(sched_proxy)
     cls.setup_populate_job_config(sched_proxy)
     sched_proxy.restartShards.return_value = cls.create_simple_success_response()
@@ -85,12 +85,12 @@ class TestRestartCommand(AuroraClientCommandTest):
   def test_restart_simple(self):
     # Test the client-side restart logic in its simplest case: everything succeeds
     mock_options = self.setup_mock_options()
-    (mock_api, mock_scheduler) = self.create_mock_api()
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
     mock_health_check = self.setup_health_checks(mock_api)
     self.setup_mock_scheduler_for_simple_restart(mock_api)
     with contextlib.nested(
         patch('twitter.common.app.get_options', return_value=mock_options),
-        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler),
+        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
         patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS),
         patch('apache.aurora.client.api.instance_watcher.InstanceWatcherHealthCheck',
             return_value=mock_health_check),
@@ -107,24 +107,24 @@ class TestRestartCommand(AuroraClientCommandTest):
         # Like the update test, the exact number of calls here doesn't matter.
         # what matters is that it must have been called once before batching, plus
         # at least once per batch, and there are 4 batches.
-        assert mock_scheduler.getTasksStatus.call_count >= 4
+        assert mock_scheduler_proxy.getTasksStatus.call_count >= 4
         # called once per batch
-        assert mock_scheduler.restartShards.call_count == 4
+        assert mock_scheduler_proxy.restartShards.call_count == 4
         # parameters for all calls are generated by the same code, so we just check one
-        mock_scheduler.restartShards.assert_called_with(JobKey(environment=self.TEST_ENV,
+        mock_scheduler_proxy.restartShards.assert_called_with(JobKey(environment=self.TEST_ENV,
             role=self.TEST_ROLE, name=self.TEST_JOB), [15, 16, 17, 18, 19], None)
 
   def test_restart_failed_status(self):
     # Test the client-side updater logic in its simplest case: everything succeeds, and no rolling
     # updates.
     mock_options = self.setup_mock_options()
-    (mock_api, mock_scheduler) = self.create_mock_api()
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
     mock_health_check = self.setup_health_checks(mock_api)
     self.setup_mock_scheduler_for_simple_restart(mock_api)
-    mock_scheduler.getTasksStatus.return_value = self.create_error_response()
+    mock_scheduler_proxy.getTasksStatus.return_value = self.create_error_response()
     with contextlib.nested(
         patch('twitter.common.app.get_options', return_value=mock_options),
-        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler),
+        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
         patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS),
         patch('apache.aurora.client.api.instance_watcher.InstanceWatcherHealthCheck',
             return_value=mock_health_check),
@@ -137,20 +137,20 @@ class TestRestartCommand(AuroraClientCommandTest):
         fp.write(self.get_valid_config())
         fp.flush()
         self.assertRaises(SystemExit, restart, ['west/mchucarroll/test/hello'], mock_options)
-        assert mock_scheduler.getTasksStatus.call_count == 1
-        assert mock_scheduler.restartShards.call_count == 0
+        assert mock_scheduler_proxy.getTasksStatus.call_count == 1
+        assert mock_scheduler_proxy.restartShards.call_count == 0
 
   def test_restart_failed_restart(self):
     # Test the client-side updater logic in its simplest case: everything succeeds, and no rolling
     # updates.
     mock_options = self.setup_mock_options()
-    (mock_api, mock_scheduler) = self.create_mock_api()
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
     mock_health_check = self.setup_health_checks(mock_api)
     self.setup_mock_scheduler_for_simple_restart(mock_api)
-    mock_scheduler.restartShards.return_value = self.create_error_response()
+    mock_scheduler_proxy.restartShards.return_value = self.create_error_response()
     with contextlib.nested(
         patch('twitter.common.app.get_options', return_value=mock_options),
-        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler),
+        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
         patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS),
         patch('apache.aurora.client.api.instance_watcher.InstanceWatcherHealthCheck',
             return_value=mock_health_check),
@@ -163,7 +163,7 @@ class TestRestartCommand(AuroraClientCommandTest):
         fp.write(self.get_valid_config())
         fp.flush()
         self.assertRaises(SystemExit, restart, ['west/mchucarroll/test/hello'], mock_options)
-        assert mock_scheduler.getTasksStatus.call_count == 1
-        assert mock_scheduler.restartShards.call_count == 1
-        mock_scheduler.restartShards.assert_called_with(JobKey(environment=self.TEST_ENV,
+        assert mock_scheduler_proxy.getTasksStatus.call_count == 1
+        assert mock_scheduler_proxy.restartShards.call_count == 1
+        mock_scheduler_proxy.restartShards.assert_called_with(JobKey(environment=self.TEST_ENV,
             role=self.TEST_ROLE, name=self.TEST_JOB), [0, 1, 2, 3, 4], None)

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/b17bc1f9/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 0a6aaf8..b214f70 100644
--- a/src/test/python/apache/aurora/client/commands/test_run.py
+++ b/src/test/python/apache/aurora/client/commands/test_run.py
@@ -80,13 +80,13 @@ class TestRunCommand(AuroraClientCommandTest):
 
   def test_successful_run(self):
     """Test the run command."""
-    # Calls api.check_status, which calls scheduler.getJobs
+    # Calls api.check_status, which calls scheduler_proxy.getJobs
     mock_options = self.setup_mock_options()
-    (mock_api, mock_scheduler) = self.create_mock_api()
-    mock_scheduler.getTasksStatus.return_value = self.create_status_response()
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
+    mock_scheduler_proxy.getTasksStatus.return_value = self.create_status_response()
     sandbox_args = {'slave_root': '/slaveroot', 'slave_run_directory': 'slaverun'}
     with contextlib.nested(
-        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler),
+        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
         patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS),
         patch('apache.aurora.client.commands.run.CLUSTERS', new=self.TEST_CLUSTERS),
         patch('twitter.common.app.get_options', return_value=mock_options),
@@ -103,7 +103,7 @@ class TestRunCommand(AuroraClientCommandTest):
 
       # The status command sends a getTasksStatus query to the scheduler,
       # and then prints the result.
-      mock_scheduler.getTasksStatus.assert_called_with(TaskQuery(jobName='hello',
+      mock_scheduler_proxy.getTasksStatus.assert_called_with(TaskQuery(jobName='hello',
           environment='test', owner=Identity(role='mchucarroll'),
           statuses=set([ScheduleStatus.RUNNING, ScheduleStatus.KILLING, ScheduleStatus.RESTARTING,
               ScheduleStatus.PREEMPTING])))

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/b17bc1f9/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 64dc896..c9a5939 100644
--- a/src/test/python/apache/aurora/client/commands/test_ssh.py
+++ b/src/test/python/apache/aurora/client/commands/test_ssh.py
@@ -73,13 +73,12 @@ class TestSshCommand(AuroraClientCommandTest):
 
   def test_successful_ssh(self):
     """Test the ssh command."""
-    # Calls api.check_status, which calls scheduler.getJobs
     mock_options = self.setup_mock_options()
-    (mock_api, mock_scheduler) = self.create_mock_api()
-    mock_scheduler.getTasksStatus.return_value = self.create_status_response()
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
+    mock_scheduler_proxy.getTasksStatus.return_value = self.create_status_response()
     sandbox_args = {'slave_root': '/slaveroot', 'slave_run_directory': 'slaverun'}
     with contextlib.nested(
-        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler),
+        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
         patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS),
         patch('twitter.common.app.get_options', return_value=mock_options),
         patch('apache.aurora.client.api.command_runner.DistributedCommandRunner.sandbox_args',
@@ -94,7 +93,7 @@ class TestSshCommand(AuroraClientCommandTest):
 
       # The status command sends a getTasksStatus query to the scheduler,
       # and then prints the result.
-      mock_scheduler.getTasksStatus.assert_called_with(TaskQuery(jobName='hello',
+      mock_scheduler_proxy.getTasksStatus.assert_called_with(TaskQuery(jobName='hello',
           environment='test', owner=Identity(role='mchucarroll'), instanceIds=set([1]),
           statuses=set([ScheduleStatus.RUNNING, ScheduleStatus.KILLING, ScheduleStatus.RESTARTING,
               ScheduleStatus.PREEMPTING])))

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/b17bc1f9/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 9241631..66c06a2 100644
--- a/src/test/python/apache/aurora/client/commands/test_status.py
+++ b/src/test/python/apache/aurora/client/commands/test_status.py
@@ -72,12 +72,12 @@ class TestListJobs(AuroraClientCommandTest):
 
   def test_successful_status(self):
     """Test the status command."""
-    # Calls api.check_status, which calls scheduler.getJobs
+    # Calls api.check_status, which calls scheduler_proxy.getJobs
     mock_options = self.setup_mock_options()
-    (mock_api, mock_scheduler) = self.create_mock_api()
-    mock_scheduler.getTasksStatus.return_value = self.create_status_response()
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
+    mock_scheduler_proxy.getTasksStatus.return_value = self.create_status_response()
     with contextlib.nested(
-        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler),
+        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
         patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS),
         patch('twitter.common.app.get_options', return_value=mock_options)) as (
             mock_scheduler_proxy_class,
@@ -87,17 +87,17 @@ class TestListJobs(AuroraClientCommandTest):
 
       # The status command sends a getTasksStatus query to the scheduler,
       # and then prints the result.
-      mock_scheduler.getTasksStatus.assert_called_with(TaskQuery(jobName='hello',
+      mock_scheduler_proxy.getTasksStatus.assert_called_with(TaskQuery(jobName='hello',
           environment='test', owner=Identity(role='mchucarroll')))
 
   def test_unsuccessful_status(self):
     """Test the status command when the user asks the status of a job that doesn't exist."""
-    # Calls api.check_status, which calls scheduler.getJobs
+    # Calls api.check_status, which calls scheduler_proxy.getJobs
     mock_options = self.setup_mock_options()
-    (mock_api, mock_scheduler) = self.create_mock_api()
-    mock_scheduler.getTasksStatus.return_value = self.create_failed_status_response()
+    (mock_api, mock_scheduler_proxy) = self.create_mock_api()
+    mock_scheduler_proxy.getTasksStatus.return_value = self.create_failed_status_response()
     with contextlib.nested(
-        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler),
+        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
         patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS),
         patch('twitter.common.app.get_options', return_value=mock_options)) as (
             mock_scheduler_proxy_class,
@@ -105,5 +105,5 @@ class TestListJobs(AuroraClientCommandTest):
             options):
       self.assertRaises(SystemExit, status, ['west/mchucarroll/test/hello'], mock_options)
 
-      mock_scheduler.getTasksStatus.assert_called_with(TaskQuery(jobName='hello',
+      mock_scheduler_proxy.getTasksStatus.assert_called_with(TaskQuery(jobName='hello',
           environment='test', owner=Identity(role='mchucarroll')))