Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 46CCC200CD7 for ; Tue, 18 Jul 2017 00:38:05 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 451E2165ED6; Mon, 17 Jul 2017 22:38:05 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id E16C7165ED5 for ; Tue, 18 Jul 2017 00:38:03 +0200 (CEST) Received: (qmail 86194 invoked by uid 500); 17 Jul 2017 22:38:02 -0000 Mailing-List: contact commits-help@aurora.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@aurora.apache.org Delivered-To: mailing list commits@aurora.apache.org Received: (qmail 86185 invoked by uid 99); 17 Jul 2017 22:38:02 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 17 Jul 2017 22:38:02 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id BF5D3E360F; Mon, 17 Jul 2017 22:38:02 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: santhk@apache.org To: commits@aurora.apache.org Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: aurora git commit: aurora job restart request should be idempotent and retryable. Date: Mon, 17 Jul 2017 22:38:02 +0000 (UTC) archived-at: Mon, 17 Jul 2017 22:38:05 -0000 Repository: aurora Updated Branches: refs/heads/master a922b053e -> 243d6fa14 aurora job restart request should be idempotent and retryable. There was a recent change to the Aurora client to provide "at most once" instead of "at least once" retries for non-idempotent operations. See: https://github.com/apache/aurora/commit/f1e25375def5a047da97d8bdfb47a3a9101568f6 Technically, `aurora job restart` is a non-idempotent operation, thus it was not retried. However, when a transport exception occurs, the operator has to babysit simple operations like aurora job restart if it were not retried. Compared to the requests that were causing problems (admin tasks, job creating, updates, etc.), restarts in general should be retried rather than erring on the side of caution. Job restart can be divided into three steps: - 1. get instance status (getTasksWithoutConfigs) - 2. restart shards (restartShards) - 3. watch instance until healthy (getTasksWithoutConfigs) TTransport exception can be thrown at each of these step, ideally we should make __ALL__ of the steps above __idempotent__ and retryable. The only trickey part is that the `watch` logic is also used in --wait-until options of job create/add command, making this step retryable will have an impact on job create/add commands as well. In this CR, I will make the first __TWO__ steps retryable since they are self-contained in job restart command. If people are OK with this strategy, I'll make the `watch` step retryable as well. __Updates__: I made the `watch` step in `aurora job restart` retryable in the 3rd revision. Note the `InstanceWatcher` in `Restarter` relies on `StatusHelper.get_tasks` method to query the latest instance status. This method is also invoked by JobMonitor during job create/add. The solution here is to pass a `retry` flag to this method, so that it can be customized for different scenarios. __Open Question__: Currently, `aurora job create --wait-until RUNNING` fails immediately if there is any transport exception during the `wait` step. Do we need retry for the `wait` step? There are three phases where transport exception could be thrown: - Phase I: job create request was not sent to scheduler yet, we should retry the command. - Phase II: job create request was already sent to scheduler, we should not retry the command, and exit. - Phase III: job create command gets to the `wait` step, that means the job create request must have already been sent to scheduler, we should not have to retry the command. So overall, we do not retry for the `wait` step. The same principle is applicable to `job add` command. Testing Done: ./build-support/jenkins/build.sh To test the retry logic, TTranport exceptions were randomly thrown at the client side(for all api calls to the scheduler proxy, there is a 50% chance of throwing TTranport exception), aurora job restart command was issued against scheduler, and the output was like: ``` vagrant@aurora:~$ aurora job restart devcluster/vagrant/test/hello WARN] Transport error communicating with scheduler: Timed out talking to http://aurora.local:8081/api, retrying... WARN] Transport error communicating with scheduler: Timed out talking to http://aurora.local:8081/api, retrying... INFO] Performing rolling restart of job devcluster/vagrant/test/hello (instances: [0]) INFO] Restarting instances: [0] WARN] Transport error communicating with scheduler: Timed out talking to http://aurora.local:8081/api, retrying... WARN] Transport error communicating with scheduler: Timed out talking to http://aurora.local:8081/api, retrying... INFO] Watching instances: [0] WARN] Transport error communicating with scheduler: Timed out talking to http://aurora.local:8081/api, retrying... INFO] Detected RUNNING instance 0 WARN] Transport error communicating with scheduler: Timed out talking to http://aurora.local:8081/api, retrying... WARN] Transport error communicating with scheduler: Timed out talking to http://aurora.local:8081/api, retrying... WARN] Transport error communicating with scheduler: Timed out talking to http://aurora.local:8081/api, retrying... INFO] Instance 0 has been up and healthy for at least 30 seconds INFO] All instances were restarted successfully Job devcluster/vagrant/test/hello restarted successfully ``` __Updates__: To ensure that job create/add/kill commands work as expected, TTranport exceptions were randomly thrown at the client side, aurora job create/add/kill commands were issued against scheduler: __JOB CREATE__: Job create command succeeds when no transport exception was thrown. ``` vagrant@aurora:~$ aurora job create devcluster/vagrant/test/hello /vagrant/hello.aurora INFO] Creating job hello INFO] Checking status of devcluster/vagrant/test/hello Job create succeeded: job url=http://aurora.local:8081/scheduler/vagrant/test/hello ``` Job create command __retries__(getTasksWithoutConfigs) when transport exception were thrown __BEFORE__ the job create request was sent to scheduler. ``` vagrant@aurora:~$ aurora job create devcluster/vagrant/test/hello /vagrant/hello.aurora INFO] Creating job hello INFO] Checking status of devcluster/vagrant/test/hello WARN] Transport error communicating with scheduler: Timed out talking to http://aurora.local:8081/api, retrying... Job create succeeded: job url=http://aurora.local:8081/scheduler/vagrant/test/hello ``` Job create command __does NOT retry__ when transport exception were thrown __AFTER__ the job create request was sent to scheduler. ``` vagrant@aurora:~$ aurora job create devcluster/vagrant/test/hello /vagrant/hello.aurora INFO] Creating job hello Transport error communicating with scheduler during non-idempotent operation: Timed out talking to http://aurora.local:8081/api, not retrying ``` __JOB ADD__: Job add command succeeds when no transport exception was thrown. ``` vagrant@aurora:~$ aurora job add devcluster/vagrant/test/hello/0 1 INFO] Adding 1 instances to devcluster/vagrant/test/hello using the task config of instance 0 ``` Job add command __retries__(getTasksWithoutConfigs) when transport exception were thrown __BEFORE__ the job add request was sent to scheduler. ``` vagrant@aurora:~$ aurora job add devcluster/vagrant/test/hello/0 1 WARN] Transport error communicating with scheduler: Timed out talking to http://aurora.local:8081/api, retrying... INFO] Adding 1 instances to devcluster/vagrant/test/hello using the task config of instance 0 ``` Job add command __does NOT retry__ when transport exception were thrown __AFTER__ the job add request was sent to scheduler. ``` vagrant@aurora:~$ aurora job add devcluster/vagrant/test/hello/0 1 INFO] Adding 1 instances to devcluster/vagrant/test/hello using the task config of instance 0 Transport error communicating with scheduler during non-idempotent operation: Timed out talking to http://aurora.local:8081/api, not retrying ``` __JOB KILL__: Job kill command succeeds when no transport exception was thrown. ``` vagrant@aurora:~$ aurora job killall devcluster/vagrant/test/hello INFO] Killing tasks for job: devcluster/vagrant/test/hello INFO] Instances to be killed: [0] Successfully killed instances [0] Job killall succeeded ``` Job kill command __retries__(getTasksWithoutConfigs) when transport exception were thrown __BEFORE__ the job kill request was sent to scheduler. ``` vagrant@aurora:~$ aurora job killall devcluster/vagrant/test/hello WARN] Transport error communicating with scheduler: Timed out talking to http://aurora.local:8081/api, retrying... INFO] Killing tasks for job: devcluster/vagrant/test/hello INFO] Instances to be killed: [0] Successfully killed instances [0] Job killall succeeded ``` Job kill command __does NOT retry__ when transport exception were thrown __AFTER__ the job kill request was sent to scheduler. ``` vagrant@aurora:~$ aurora job killall devcluster/vagrant/test/hello INFO] Killing tasks for job: devcluster/vagrant/test/hello INFO] Instances to be killed: [0] Transport error communicating with scheduler during non-idempotent operation: Timed out talking to http://aurora.local:8081/api, not retrying ``` Bugs closed: AURORA-1940 Reviewed at https://reviews.apache.org/r/60714/ Project: http://git-wip-us.apache.org/repos/asf/aurora/repo Commit: http://git-wip-us.apache.org/repos/asf/aurora/commit/243d6fa1 Tree: http://git-wip-us.apache.org/repos/asf/aurora/tree/243d6fa1 Diff: http://git-wip-us.apache.org/repos/asf/aurora/diff/243d6fa1 Branch: refs/heads/master Commit: 243d6fa14fb46eec79de6b1f7f55a4c1937ee8f5 Parents: a922b05 Author: Kai Huang Authored: Mon Jul 17 15:37:42 2017 -0700 Committer: Santhosh Kumar Committed: Mon Jul 17 15:37:42 2017 -0700 ---------------------------------------------------------------------- .../aurora/client/api/instance_watcher.py | 2 +- .../apache/aurora/client/api/restarter.py | 4 ++-- .../apache/aurora/client/api/task_util.py | 5 +++-- src/test/python/apache/aurora/api_util.py | 3 +++ .../aurora/client/api/test_instance_watcher.py | 9 +++++---- .../aurora/client/api/test_job_monitor.py | 8 ++++---- .../apache/aurora/client/api/test_restarter.py | 21 ++++++++++---------- .../apache/aurora/client/api/test_task_util.py | 7 ++++--- .../aurora/client/cli/test_command_hooks.py | 2 +- .../apache/aurora/client/cli/test_create.py | 2 +- .../apache/aurora/client/cli/test_plugins.py | 2 +- .../apache/aurora/client/cli/test_restart.py | 6 +++--- 12 files changed, 39 insertions(+), 32 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/aurora/blob/243d6fa1/src/main/python/apache/aurora/client/api/instance_watcher.py ---------------------------------------------------------------------- diff --git a/src/main/python/apache/aurora/client/api/instance_watcher.py b/src/main/python/apache/aurora/client/api/instance_watcher.py index a35fb22..07ac001 100644 --- a/src/main/python/apache/aurora/client/api/instance_watcher.py +++ b/src/main/python/apache/aurora/client/api/instance_watcher.py @@ -90,7 +90,7 @@ class InstanceWatcher(object): instance_states[instance_id] = Instance(finished=True) while not self._terminating.is_set(): - running_tasks = self._status_helper.get_tasks(instance_ids) + running_tasks = self._status_helper.get_tasks(instance_ids, retry=True) now = self._clock.time() tasks_by_instance = dict((task.assignedTask.instanceId, task) for task in running_tasks) for instance_id in instance_ids: http://git-wip-us.apache.org/repos/asf/aurora/blob/243d6fa1/src/main/python/apache/aurora/client/api/restarter.py ---------------------------------------------------------------------- diff --git a/src/main/python/apache/aurora/client/api/restarter.py b/src/main/python/apache/aurora/client/api/restarter.py index 6600c6b..4523db8 100644 --- a/src/main/python/apache/aurora/client/api/restarter.py +++ b/src/main/python/apache/aurora/client/api/restarter.py @@ -60,7 +60,7 @@ class Restarter(object): # Verify that this operates on a valid job. query = self._job_key.to_thrift_query() query.statuses = ACTIVE_STATES - status = self._scheduler.getTasksWithoutConfigs(query) + status = self._scheduler.getTasksWithoutConfigs(query, retry=True) if status.responseCode != ResponseCode.OK: return status @@ -85,7 +85,7 @@ class Restarter(object): log.info("Restarting instances: %s", batch) - resp = self._scheduler.restartShards(self._job_key.to_thrift(), batch) + resp = self._scheduler.restartShards(self._job_key.to_thrift(), batch, retry=True) if resp.responseCode != ResponseCode.OK: log.error('Error restarting instances: %s', combine_messages(resp)) return resp http://git-wip-us.apache.org/repos/asf/aurora/blob/243d6fa1/src/main/python/apache/aurora/client/api/task_util.py ---------------------------------------------------------------------- diff --git a/src/main/python/apache/aurora/client/api/task_util.py b/src/main/python/apache/aurora/client/api/task_util.py index fb7c76f..32dc00e 100644 --- a/src/main/python/apache/aurora/client/api/task_util.py +++ b/src/main/python/apache/aurora/client/api/task_util.py @@ -25,17 +25,18 @@ class StatusHelper(object): self._scheduler = scheduler self._query_factory = query_factory - def get_tasks(self, instance_ids=None): + def get_tasks(self, instance_ids=None, retry=False): """Gets tasks from the scheduler. Arguments: instance_ids -- optional list of instance IDs to query for. + retry -- optional boolean value indicating whether to retry the operation. Returns a list of tasks. """ log.debug('Querying instance statuses: %s' % instance_ids) try: - resp = self._scheduler.getTasksWithoutConfigs(self._query_factory(instance_ids)) + resp = self._scheduler.getTasksWithoutConfigs(self._query_factory(instance_ids), retry=retry) except IOError as e: log.error('IO Exception during scheduler call: %s' % e) return [] http://git-wip-us.apache.org/repos/asf/aurora/blob/243d6fa1/src/test/python/apache/aurora/api_util.py ---------------------------------------------------------------------- diff --git a/src/test/python/apache/aurora/api_util.py b/src/test/python/apache/aurora/api_util.py index bd6e3a6..41441f3 100644 --- a/src/test/python/apache/aurora/api_util.py +++ b/src/test/python/apache/aurora/api_util.py @@ -151,5 +151,8 @@ class SchedulerProxyApiSpec(SchedulerThriftApiSpec, SchedulerProxy): def startJobUpdate(self, request, message, retry=True): pass + def restartShards(self, job, shardIds, retry=True): + pass + def url(self): pass http://git-wip-us.apache.org/repos/asf/aurora/blob/243d6fa1/src/test/python/apache/aurora/client/api/test_instance_watcher.py ---------------------------------------------------------------------- diff --git a/src/test/python/apache/aurora/client/api/test_instance_watcher.py b/src/test/python/apache/aurora/client/api/test_instance_watcher.py index 8fd419f..ec1686e 100644 --- a/src/test/python/apache/aurora/client/api/test_instance_watcher.py +++ b/src/test/python/apache/aurora/client/api/test_instance_watcher.py @@ -20,7 +20,8 @@ import mox from apache.aurora.client.api.health_check import HealthCheck from apache.aurora.client.api.instance_watcher import InstanceWatcher -from gen.apache.aurora.api.AuroraSchedulerManager import Client as scheduler_client +from ...api_util import SchedulerProxyApiSpec + from gen.apache.aurora.api.ttypes import ( AssignedTask, JobKey, @@ -74,7 +75,7 @@ class InstanceWatcherTest(unittest.TestCase): def setUp(self): self._clock = FakeClock() self._event = FakeEvent(self._clock) - self._scheduler = mox.MockObject(scheduler_client) + self._scheduler = mox.MockObject(SchedulerProxyApiSpec) self._job_key = JobKey(role='mesos', name='jimbob', environment='test') self._health_check = mox.MockObject(HealthCheck) self._watcher = InstanceWatcher(self._scheduler, @@ -104,14 +105,14 @@ class InstanceWatcherTest(unittest.TestCase): query = self.get_tasks_status_query(instance_ids) for _ in range(int(num_calls)): - self._scheduler.getTasksWithoutConfigs(query).AndReturn(response) + self._scheduler.getTasksWithoutConfigs(query, retry=True).AndReturn(response) def expect_io_error_in_get_statuses(self, instance_ids=WATCH_INSTANCES, num_calls=EXPECTED_CYCLES): query = self.get_tasks_status_query(instance_ids) for _ in range(int(num_calls)): - self._scheduler.getTasksWithoutConfigs(query).AndRaise(IOError('oops')) + self._scheduler.getTasksWithoutConfigs(query, retry=True).AndRaise(IOError('oops')) def mock_health_check(self, task, status): self._health_check.health(task).InAnyOrder().AndReturn(status) http://git-wip-us.apache.org/repos/asf/aurora/blob/243d6fa1/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 537abd3..fb32caf 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 @@ -19,7 +19,7 @@ from mock import create_autospec from apache.aurora.client.api.job_monitor import JobMonitor from apache.aurora.common.aurora_job_key import AuroraJobKey -from ...api_util import SchedulerThriftApiSpec +from ...api_util import SchedulerProxyApiSpec from gen.apache.aurora.api.ttypes import ( AssignedTask, @@ -53,7 +53,7 @@ class FakeEvent(object): class JobMonitorTest(unittest.TestCase): def setUp(self): - self._scheduler = create_autospec(spec=SchedulerThriftApiSpec, instance=True) + self._scheduler = create_autospec(spec=SchedulerProxyApiSpec, instance=True) self._job_key = AuroraJobKey('cl', 'johndoe', 'test', 'test_job') self._event = FakeEvent() @@ -81,9 +81,9 @@ class JobMonitorTest(unittest.TestCase): query.instanceIds = frozenset([int(s) for s in instances]) if once: - self._scheduler.getTasksWithoutConfigs.assert_called_once_with(query) + self._scheduler.getTasksWithoutConfigs.assert_called_once_with(query, retry=False) else: - self._scheduler.getTasksWithoutConfigs.assert_called_with(query) + self._scheduler.getTasksWithoutConfigs.assert_called_with(query, retry=False) def test_wait_until_state(self): self.mock_get_tasks([ http://git-wip-us.apache.org/repos/asf/aurora/blob/243d6fa1/src/test/python/apache/aurora/client/api/test_restarter.py ---------------------------------------------------------------------- diff --git a/src/test/python/apache/aurora/client/api/test_restarter.py b/src/test/python/apache/aurora/client/api/test_restarter.py index a81003e..9561715 100644 --- a/src/test/python/apache/aurora/client/api/test_restarter.py +++ b/src/test/python/apache/aurora/client/api/test_restarter.py @@ -17,11 +17,9 @@ from mox import IgnoreArg, MoxTestBase from apache.aurora.client.api.instance_watcher import InstanceWatcher from apache.aurora.client.api.restarter import Restarter, RestartSettings from apache.aurora.common.aurora_job_key import AuroraJobKey -from apache.aurora.common.cluster import Cluster -from ..fake_scheduler_proxy import FakeSchedulerProxy +from ...api_util import SchedulerProxyApiSpec -from gen.apache.aurora.api.AuroraSchedulerManager import Client as scheduler_client from gen.apache.aurora.api.ttypes import ( AssignedTask, Response, @@ -58,17 +56,18 @@ class TestRestarter(MoxTestBase): def setUp(self): super(TestRestarter, self).setUp() - self.mock_scheduler = self.mox.CreateMock(scheduler_client) self.mock_instance_watcher = self.mox.CreateMock(InstanceWatcher) + self.mock_scheduler_proxy = self.mox.CreateMock(SchedulerProxyApiSpec) self.restarter = Restarter( JOB, RESTART_SETTINGS, - FakeSchedulerProxy(Cluster(name=CLUSTER), self.mock_scheduler), + self.mock_scheduler_proxy, self.mock_instance_watcher) def mock_restart_instances(self, instances): - self.mock_scheduler.restartShards(JOB.to_thrift(), instances).AndReturn(make_response()) + self.mock_scheduler_proxy.restartShards(JOB.to_thrift(), instances, retry=True).AndReturn( + make_response()) self.mock_instance_watcher.watch(instances).AndReturn([]) def test_restart_one_iteration(self): @@ -100,7 +99,7 @@ class TestRestarter(MoxTestBase): assignedTask=AssignedTask(task=TaskConfig(), instanceId=i) )) response = make_response(result=Result(scheduleStatusResult=ScheduleStatusResult(tasks=tasks))) - self.mock_scheduler.getTasksWithoutConfigs(IgnoreArg()).AndReturn(response) + self.mock_scheduler_proxy.getTasksWithoutConfigs(IgnoreArg(), retry=True).AndReturn(response) def test_restart_all_instances(self): self.mock_status_active_tasks([0, 1, 3, 4, 5]) @@ -112,7 +111,7 @@ class TestRestarter(MoxTestBase): def mock_status_no_active_task(self): response = make_response(code=ResponseCode.INVALID_REQUEST) - self.mock_scheduler.getTasksWithoutConfigs(IgnoreArg()).AndReturn(response) + self.mock_scheduler_proxy.getTasksWithoutConfigs(IgnoreArg(), retry=True).AndReturn(response) def test_restart_no_instance_active(self): self.mock_status_no_active_task() @@ -123,7 +122,8 @@ class TestRestarter(MoxTestBase): def mock_restart_fails(self): response = make_response(code=ResponseCode.ERROR, message='test error') - self.mock_scheduler.restartShards(JOB.to_thrift(), IgnoreArg()).AndReturn(response) + self.mock_scheduler_proxy.restartShards(JOB.to_thrift(), IgnoreArg(), retry=True).AndReturn( + response) def test_restart_instance_fails(self): self.mock_status_active_tasks([0, 1]) @@ -134,7 +134,8 @@ class TestRestarter(MoxTestBase): assert self.restarter.restart(None).responseCode == ResponseCode.ERROR def mock_restart_watch_fails(self, instances): - self.mock_scheduler.restartShards(JOB.to_thrift(), instances).AndReturn(make_response()) + self.mock_scheduler_proxy.restartShards(JOB.to_thrift(), instances, retry=True).AndReturn( + make_response()) self.mock_instance_watcher.watch(instances).AndReturn(instances) def test_restart_instances_watch_fails(self): http://git-wip-us.apache.org/repos/asf/aurora/blob/243d6fa1/src/test/python/apache/aurora/client/api/test_task_util.py ---------------------------------------------------------------------- diff --git a/src/test/python/apache/aurora/client/api/test_task_util.py b/src/test/python/apache/aurora/client/api/test_task_util.py index 365ef59..8941378 100644 --- a/src/test/python/apache/aurora/client/api/test_task_util.py +++ b/src/test/python/apache/aurora/client/api/test_task_util.py @@ -18,7 +18,7 @@ from mock import call, create_autospec from apache.aurora.client.api.task_util import StatusHelper -from ...api_util import SchedulerThriftApiSpec +from ...api_util import SchedulerProxyApiSpec from gen.apache.aurora.api.ttypes import ( AssignedTask, @@ -51,7 +51,7 @@ class TaskUtilTest(unittest.TestCase): @classmethod def mock_scheduler(cls, response_code=None): - scheduler = create_autospec(spec=SchedulerThriftApiSpec, instance=True) + scheduler = create_autospec(spec=SchedulerProxyApiSpec, instance=True) response_code = ResponseCode.OK if response_code is None else response_code resp = Response(responseCode=response_code, details=[ResponseDetail(message='test')]) resp.result = Result(scheduleStatusResult=ScheduleStatusResult(tasks=cls.create_tasks())) @@ -63,5 +63,6 @@ class TaskUtilTest(unittest.TestCase): helper = self.create_helper(scheduler, self.create_query) tasks = helper.get_tasks(self.INSTANCES) - assert scheduler.getTasksWithoutConfigs.mock_calls == [call(self.create_query(self.INSTANCES))] + assert scheduler.getTasksWithoutConfigs.mock_calls == [call(self.create_query( + self.INSTANCES), retry=False)] assert 1 == len(tasks) http://git-wip-us.apache.org/repos/asf/aurora/blob/243d6fa1/src/test/python/apache/aurora/client/cli/test_command_hooks.py ---------------------------------------------------------------------- diff --git a/src/test/python/apache/aurora/client/cli/test_command_hooks.py b/src/test/python/apache/aurora/client/cli/test_command_hooks.py index a44a25f..8b2760d 100644 --- a/src/test/python/apache/aurora/client/cli/test_command_hooks.py +++ b/src/test/python/apache/aurora/client/cli/test_command_hooks.py @@ -94,7 +94,7 @@ class TestClientCreateCommand(AuroraClientCommandTest): @classmethod def assert_scheduler_called(cls, mock_api, mock_query, num_queries): assert mock_api.scheduler_proxy.getTasksWithoutConfigs.call_count == num_queries - mock_api.scheduler_proxy.getTasksWithoutConfigs.assert_called_with(mock_query) + mock_api.scheduler_proxy.getTasksWithoutConfigs.assert_called_with(mock_query, retry=False) def test_create_job_with_successful_hook(self): GlobalCommandHookRegistry.reset() http://git-wip-us.apache.org/repos/asf/aurora/blob/243d6fa1/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 3b09bb2..e029ada 100644 --- a/src/test/python/apache/aurora/client/cli/test_create.py +++ b/src/test/python/apache/aurora/client/cli/test_create.py @@ -110,7 +110,7 @@ class TestClientCreateCommand(AuroraClientCommandTest): @classmethod def assert_scheduler_called(cls, mock_api, mock_query, num_queries): assert mock_api.scheduler_proxy.getTasksWithoutConfigs.call_count == num_queries - mock_api.scheduler_proxy.getTasksWithoutConfigs.assert_called_with(mock_query) + mock_api.scheduler_proxy.getTasksWithoutConfigs.assert_called_with(mock_query, retry=False) def test_simple_successful_create_job(self): """Run a test of the "create" command against a mocked-out API: http://git-wip-us.apache.org/repos/asf/aurora/blob/243d6fa1/src/test/python/apache/aurora/client/cli/test_plugins.py ---------------------------------------------------------------------- diff --git a/src/test/python/apache/aurora/client/cli/test_plugins.py b/src/test/python/apache/aurora/client/cli/test_plugins.py index 762735e..6a244a9 100644 --- a/src/test/python/apache/aurora/client/cli/test_plugins.py +++ b/src/test/python/apache/aurora/client/cli/test_plugins.py @@ -89,7 +89,7 @@ class TestPlugins(AuroraClientCommandTest): @classmethod def assert_scheduler_called(cls, mock_api, mock_query, num_queries): assert mock_api.scheduler_proxy.getTasksWithoutConfigs.call_count == num_queries - mock_api.scheduler_proxy.getTasksWithoutConfigs.assert_called_with(mock_query) + mock_api.scheduler_proxy.getTasksWithoutConfigs.assert_called_with(mock_query, retry=False) def test_plugin_runs_in_create_job(self): """Run a test of the "create" command against a mocked-out API: http://git-wip-us.apache.org/repos/asf/aurora/blob/243d6fa1/src/test/python/apache/aurora/client/cli/test_restart.py ---------------------------------------------------------------------- diff --git a/src/test/python/apache/aurora/client/cli/test_restart.py b/src/test/python/apache/aurora/client/cli/test_restart.py index cb4adc5..b9731dc 100644 --- a/src/test/python/apache/aurora/client/cli/test_restart.py +++ b/src/test/python/apache/aurora/client/cli/test_restart.py @@ -121,7 +121,7 @@ class TestRestartCommand(AuroraClientCommandTest): 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_proxy.restartShards.assert_called_with(JobKey(environment=self.TEST_ENV, - role=self.TEST_ROLE, name=self.TEST_JOB), [15, 16, 17, 18, 19]) + role=self.TEST_ROLE, name=self.TEST_JOB), [15, 16, 17, 18, 19], retry=True) def test_restart_simple_no_config(self): # Test the client-side restart logic in its simplest case: everything succeeds @@ -141,7 +141,7 @@ class TestRestartCommand(AuroraClientCommandTest): assert mock_scheduler_proxy.getTasksWithoutConfigs.call_count >= 4 assert mock_scheduler_proxy.restartShards.call_count == 4 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]) + role=self.TEST_ROLE, name=self.TEST_JOB), [15, 16, 17, 18, 19], retry=True) def test_restart_invalid_shards(self): # Test the client-side restart when a shard argument is too large, and it's @@ -242,7 +242,7 @@ class TestRestartCommand(AuroraClientCommandTest): assert mock_scheduler_proxy.getTasksWithoutConfigs.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]) + role=self.TEST_ROLE, name=self.TEST_JOB), [0, 1, 2, 3, 4], retry=True) assert result == EXIT_API_ERROR MOCK_OUT = []