Scheduler test app manager

As a preparation for scale-out-scheduler the scheduler in tests
were extracted in order to start multiple instances in a previous
change.

This change continues on by introducing a manager to create
additional scheduler instances and the ability to call certain
methods on some or all of those instances.

This change only touches tests.

Change-Id: Ia05a7221f19bad97de1176239c075b8fc9dab7e5
Story: 2007192
This commit is contained in:
Jan Kubovy 2020-02-28 10:44:41 +01:00
parent 0437c5fa6c
commit a770be9b83
12 changed files with 143 additions and 99 deletions

View File

@ -30,6 +30,7 @@ import queue
import random
import re
from logging import Logger
from typing import Callable, Optional, Any, Iterable
import requests
import select
@ -90,8 +91,7 @@ import zuul.configloader
from zuul.lib.config import get_default
from zuul.lib.logutil import get_annotated_logger
FIXTURE_DIR = os.path.join(os.path.dirname(__file__),
'fixtures')
FIXTURE_DIR = os.path.join(os.path.dirname(__file__), 'fixtures')
KEEP_TEMPDIRS = bool(os.environ.get('KEEP_TEMPDIRS', False))
@ -3436,20 +3436,20 @@ class SchedulerTestApp:
self.sched.registerConnections(connections)
self.executor_client = zuul.executor.client.ExecutorClient(
executor_client = zuul.executor.client.ExecutorClient(
self.config, self.sched)
self.merge_client = RecordingMergeClient(self.config, self.sched)
self.nodepool = zuul.nodepool.Nodepool(self.sched)
self.zk = zuul.zk.ZooKeeper(enable_cache=True)
self.zk.connect(self.zk_config, timeout=60.0)
merge_client = RecordingMergeClient(self.config, self.sched)
nodepool = zuul.nodepool.Nodepool(self.sched)
zk = zuul.zk.ZooKeeper(enable_cache=True)
zk.connect(self.zk_config, timeout=30.0)
self.sched.setExecutor(self.executor_client)
self.sched.setMerger(self.merge_client)
self.sched.setNodepool(self.nodepool)
self.sched.setZooKeeper(self.zk)
self.sched.setExecutor(executor_client)
self.sched.setMerger(merge_client)
self.sched.setNodepool(nodepool)
self.sched.setZooKeeper(zk)
self.sched.start()
self.executor_client.gearman.waitForServer()
executor_client.gearman.waitForServer()
self.sched.reconfigure(self.config)
self.sched.resume()
@ -3472,6 +3472,46 @@ class SchedulerTestApp:
self.log.exception("Reconfiguration failed:")
class SchedulerTestManager:
def __init__(self):
self.instances = []
def create(self, log: Logger, config: ConfigParser, zk_config: str,
connections: ConnectionRegistry) -> SchedulerTestApp:
app = SchedulerTestApp(log, config, zk_config, connections)
self.instances.append(app)
return app
def __len__(self) -> int:
return len(self.instances)
def __getitem__(self, item: int) -> SchedulerTestApp:
return self.instances[item]
def __setitem__(self, key: int, value: SchedulerTestApp):
raise Exception("Not implemented, use create method!")
def __delitem__(self, key, value):
raise Exception("Not implemented!")
def __iter__(self):
return iter(self.instances)
def filter(self, matcher=None) -> Iterable[SchedulerTestApp]:
fcn = None # type: Optional[Callable[[int, SchedulerTestApp], bool]]
if type(matcher) == list:
def fcn(_: int, app: SchedulerTestApp) -> bool:
return app in matcher
elif type(matcher).__name__ == 'function':
fcn = matcher
return [e[1] for e in enumerate(self.instances)
if fcn is None or fcn(e[0], e[1])]
def execute(self, function: Callable[[Any], None], matcher=None) -> None:
for instance in self.filter(matcher):
function(instance)
class ZuulTestCase(BaseTestCase):
"""A test case with a functioning Zuul.
@ -3660,21 +3700,21 @@ class ZuulTestCase(BaseTestCase):
self.history = self.executor_server.build_history
self.builds = self.executor_server.running_builds
self.sched_app = SchedulerTestApp(self.log, self.config,
self.zk_config,
self.connections)
self.sched = self.sched_app.sched
self.event_queues = self.sched_app.event_queues + self.event_queues
self.scheds = SchedulerTestManager()
sched_app = self.scheds.create(
self.log, self.config, self.zk_config, self.connections)
self.sched = sched_app.sched
self.event_queues = sched_app.event_queues + self.event_queues
if hasattr(self, 'fake_github'):
self.event_queues.append(
self.fake_github.github_event_connector._event_forward_queue)
self.executor_client = self.sched_app.executor_client
self.merge_client = self.sched_app.merge_client
self.executor_client = sched_app.sched.executor
self.merge_client = sched_app.sched.merger
self.merge_server = None
self.nodepool = self.sched_app.nodepool
self.zk = self.sched_app.zk
self.nodepool = sched_app.sched.nodepool
self.zk = sched_app.sched.zk
# Cleanups are run in reverse order
self.addCleanup(self.assertCleanShutdown)
@ -4075,8 +4115,8 @@ class ZuulTestCase(BaseTestCase):
self.merge_server.join()
self.executor_server.stop()
self.executor_server.join()
self.sched.stop()
self.sched.join()
self.scheds.execute(lambda app: app.sched.stop())
self.scheds.execute(lambda app: app.sched.join())
self.statsd.stop()
self.statsd.join()
self.rpcclient.shutdown()
@ -4279,7 +4319,7 @@ class ZuulTestCase(BaseTestCase):
def areAllNodeRequestsComplete(self):
if self.fake_nodepool.paused:
return True
if self.sched.nodepool.requests:
if self.nodepool.requests:
return False
return True
@ -4338,7 +4378,8 @@ class ZuulTestCase(BaseTestCase):
# Join ensures that the queue is empty _and_ events have been
# processed
self.eventQueuesJoin()
self.sched.run_handler_lock.acquire()
self.scheds.execute(
lambda app: app.sched.run_handler_lock.acquire())
if (self.areAllMergeJobsWaiting() and
self.haveAllBuildsReported() and
self.areAllBuildsWaiting() and
@ -4349,14 +4390,16 @@ class ZuulTestCase(BaseTestCase):
# when locked the run handler and checked that the
# components were stable, we don't erroneously
# report that we are settled.
self.sched.run_handler_lock.release()
self.scheds.execute(
lambda app: app.sched.run_handler_lock.release())
self.executor_server.lock.release()
self.log.debug("...settled. (%s)", msg)
self.logState()
return
self.sched.run_handler_lock.release()
self.scheds.execute(
lambda app: app.sched.run_handler_lock.release())
self.executor_server.lock.release()
self.sched.wake_event.wait(0.1)
self.scheds.execute(lambda app: app.sched.wake_event.wait(0.1))
def waitForPoll(self, poller, timeout=30):
self.log.debug("Wait for poll on %s", poller)

View File

@ -308,7 +308,7 @@ class TestConnectionsBadSQL(ZuulDBTestCase):
"Test the SQL reporter fails gracefully when unable to connect"
self.config.set('zuul', 'layout_config',
'tests/fixtures/layout-sql-reporter.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
# Trigger a reporter. If no errors are raised, the reporter has been
# disabled correctly

View File

@ -346,7 +346,7 @@ class TestGerritToGithubCRD(ZuulTestCase):
self.fake_gerrit.addEvent(A.getPatchsetCreatedEvent(1))
self.waitUntilSettled()
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
# Make sure the items still share a change queue, and the
# first one is not live.
@ -795,7 +795,7 @@ class TestGithubToGerritCRD(ZuulTestCase):
self.fake_github.emitEvent(A.getPullRequestEditedEvent())
self.waitUntilSettled()
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
# Make sure the items still share a change queue, and the
# first one is not live.

View File

@ -281,7 +281,7 @@ class TestExecutorRepos(ZuulTestCase):
# Start timer trigger - also org/project
self.commitConfigUpdate('common-config',
'layouts/repo-checkout-timer-override.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
# The pipeline triggers every second, so we should have seen
# several by now.
@ -292,7 +292,7 @@ class TestExecutorRepos(ZuulTestCase):
# below don't race against more jobs being queued.
self.commitConfigUpdate('common-config',
'layouts/repo-checkout-no-timer-override.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
# If APScheduler is in mid-event when we remove the job, we
# can end up with one more event firing, so give it an extra
@ -320,7 +320,7 @@ class TestExecutorRepos(ZuulTestCase):
# Start timer trigger - also org/project
self.commitConfigUpdate('common-config',
'layouts/repo-checkout-timer.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
p1 = 'review.example.com/org/project1'
projects = [p1]
@ -339,7 +339,7 @@ class TestExecutorRepos(ZuulTestCase):
# below don't race against more jobs being queued.
self.commitConfigUpdate('common-config',
'layouts/repo-checkout-no-timer.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
# If APScheduler is in mid-event when we remove the job, we
# can end up with one more event firing, so give it an extra

View File

@ -345,7 +345,7 @@ class TestChecksApi(ZuulTestCase):
self.fake_gerrit.addFakeChecker(uuid='zuul_check:abcd',
repository='org/project',
status='ENABLED')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
A = self.fake_gerrit.addFakeChange('org/project', 'master', 'A')

View File

@ -516,7 +516,7 @@ class TestGerritCRD(ZuulTestCase):
self.fake_gerrit.addEvent(A.getPatchsetCreatedEvent(1))
self.waitUntilSettled()
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
# Make sure the items still share a change queue, and the
# first one is not live.

View File

@ -445,7 +445,7 @@ class TestGerritLegacyCRD(ZuulTestCase):
self.fake_gerrit.addEvent(A.getPatchsetCreatedEvent(1))
self.waitUntilSettled()
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
# Make sure the items still share a change queue, and the
# first one is not live.

View File

@ -147,7 +147,7 @@ class TestGithubDriver(ZuulTestCase):
self.waitUntilSettled()
# Trigger reconfig to enforce a reenqueue of the item
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
# Now we can release all jobs
@ -324,7 +324,7 @@ class TestGithubDriver(ZuulTestCase):
self.executor_server.hold_jobs_in_build = True
self.commitConfigUpdate('org/common-config',
'layouts/timer-github.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
time.sleep(2)
self.waitUntilSettled()
self.assertEqual(len(self.builds), 1)
@ -333,7 +333,7 @@ class TestGithubDriver(ZuulTestCase):
# below don't race against more jobs being queued.
self.commitConfigUpdate('org/common-config',
'layouts/basic-github.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
# If APScheduler is in mid-event when we remove the job, we
# can end up with one more event firing, so give it an extra
@ -1240,7 +1240,7 @@ class TestGithubUnprotectedBranches(ZuulTestCase):
github = self.fake_github.getGithubClient()
repo = github.repo_from_project('org/project2')
repo._set_branch_protection('master', True)
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
tenant = self.sched.abide.tenants.get('tenant-one')
@ -1270,7 +1270,7 @@ class TestGithubUnprotectedBranches(ZuulTestCase):
self.create_branch('org/project3', 'stable')
repo._set_branch_protection('stable', True)
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
A = self.fake_github.openFakePullRequest('org/project3', 'stable', 'A')
@ -1305,7 +1305,7 @@ class TestGithubUnprotectedBranches(ZuulTestCase):
repo = github.repo_from_project('org/project1')
self.create_branch('org/project1', 'feat-x')
repo._set_branch_protection('master', True)
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
A = self.fake_github.openFakePullRequest('org/project1', 'master', 'A')
@ -1376,7 +1376,7 @@ class TestGithubUnprotectedBranches(ZuulTestCase):
# deleted.
repo._create_branch('feat-x')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
# record previous tenant reconfiguration time, which may not be set
@ -1411,7 +1411,7 @@ class TestGithubUnprotectedBranches(ZuulTestCase):
repo._create_branch('release')
repo._create_branch('feature')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
repo._set_branch_protection('release', True)

View File

@ -2190,7 +2190,7 @@ class TestScheduler(ZuulTestCase):
self.wait_timeout = 120
"test that dependent changes behind dequeued changes work"
# This complicated test is a reproduction of a real life bug
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.executor_server.hold_jobs_in_build = True
A = self.fake_gerrit.addFakeChange('org/project1', 'master', 'A')
@ -2603,7 +2603,7 @@ class TestScheduler(ZuulTestCase):
self.executor_server.hold_jobs_in_build = True
# Start timer trigger - also org/project
self.commitConfigUpdate('common-config', 'layouts/idle.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
# The pipeline triggers every second, so we should have seen
# several by now.
time.sleep(5)
@ -2612,7 +2612,7 @@ class TestScheduler(ZuulTestCase):
# below don't race against more jobs being queued.
# Must be in same repo, so overwrite config with another one
self.commitConfigUpdate('common-config', 'layouts/no-timer.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
# If APScheduler is in mid-event when we remove the job, we
# can end up with one more event firing, so give it an extra
@ -2900,7 +2900,7 @@ class TestScheduler(ZuulTestCase):
self.assertEqual(len(self.gearman_server.getQueue()), 1)
self.commitConfigUpdate('common-config', 'layouts/no-jobs.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
self.gearman_server.release('gate-noop')
@ -3248,7 +3248,7 @@ class TestScheduler(ZuulTestCase):
self.fake_gerrit.addEvent(A.addApproval('Approved', 1))
self.waitUntilSettled()
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
self.executor_server.hold_jobs_in_build = False
@ -3304,7 +3304,7 @@ class TestScheduler(ZuulTestCase):
# reconfiguration.
change.branch = None
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
self.executor_server.hold_jobs_in_build = False
@ -3353,7 +3353,7 @@ class TestScheduler(ZuulTestCase):
# Add the "project-test3" job.
self.commitConfigUpdate('common-config',
'layouts/live-reconfiguration-add-job.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
self.executor_server.hold_jobs_in_build = False
@ -3414,7 +3414,7 @@ class TestScheduler(ZuulTestCase):
# Add the "project-test3" job.
self.commitConfigUpdate('common-config',
'layouts/live-reconfiguration-add-job.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
self.executor_server.hold_jobs_in_build = False
@ -3467,7 +3467,7 @@ class TestScheduler(ZuulTestCase):
# Remove the test1 job.
self.commitConfigUpdate('common-config',
'layouts/live-reconfiguration-failed-job.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
self.executor_server.hold_jobs_in_build = False
@ -3517,7 +3517,7 @@ class TestScheduler(ZuulTestCase):
self.commitConfigUpdate(
'common-config',
'layouts/live-reconfiguration-shared-queue.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
self.executor_server.hold_jobs_in_build = False
@ -3559,7 +3559,7 @@ class TestScheduler(ZuulTestCase):
self.commitConfigUpdate(
'common-config',
'layouts/live-reconfiguration-shared-queue-removed.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
self.executor_server.hold_jobs_in_build = False
@ -3592,7 +3592,7 @@ class TestScheduler(ZuulTestCase):
self.waitUntilSettled()
# Reconfigure (with only one change in the pipeline).
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
# Add the child change.
@ -3602,7 +3602,7 @@ class TestScheduler(ZuulTestCase):
self.waitUntilSettled()
# Reconfigure (with both in the pipeline).
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
self.executor_server.hold_jobs_in_build = False
@ -3641,7 +3641,7 @@ class TestScheduler(ZuulTestCase):
self.commitConfigUpdate(
'common-config',
'layouts/live-reconfiguration-del-project.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
# Builds for C aborted, builds for A succeed,
@ -3711,7 +3711,7 @@ class TestScheduler(ZuulTestCase):
dict(name='job2', result='ABORTED', changes='1,1 2,1'),
], ordered=False)
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
self.executor_server.hold_jobs_in_build = False
@ -3742,7 +3742,7 @@ class TestScheduler(ZuulTestCase):
self.commitConfigUpdate(
'common-config',
'layouts/delayed-repo-init.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
A = self.fake_gerrit.addFakeChange('org/new-project', 'master', 'A')
@ -3872,7 +3872,7 @@ class TestScheduler(ZuulTestCase):
self.create_branch('org/project', 'stable')
self.executor_server.hold_jobs_in_build = True
self.commitConfigUpdate('common-config', 'layouts/timer-template.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
# The pipeline triggers every second, so we should have seen
# several by now.
@ -3897,7 +3897,7 @@ class TestScheduler(ZuulTestCase):
# Stop queuing timer triggered jobs so that the assertions
# below don't race against more jobs being queued.
self.commitConfigUpdate('common-config', 'layouts/no-timer.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
# If APScheduler is in mid-event when we remove the job, we
# can end up with one more event firing, so give it an extra
@ -3922,7 +3922,7 @@ class TestScheduler(ZuulTestCase):
self.create_branch('org/project', 'stable')
self.executor_server.hold_jobs_in_build = True
self.commitConfigUpdate('common-config', config_file)
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
# The pipeline triggers every second, so we should have seen
# several by now.
@ -3957,7 +3957,7 @@ class TestScheduler(ZuulTestCase):
# Stop queuing timer triggered jobs so that the assertions
# below don't race against more jobs being queued.
self.commitConfigUpdate('common-config', 'layouts/no-timer.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
# If APScheduler is in mid-event when we remove the job, we
# can end up with one more event firing, so give it an extra
@ -3996,7 +3996,7 @@ class TestScheduler(ZuulTestCase):
# Start timer trigger
self.commitConfigUpdate('common-config',
'layouts/idle.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
# The pipeline triggers every second, so we should have seen
@ -4007,7 +4007,7 @@ class TestScheduler(ZuulTestCase):
# below don't race against more jobs being queued.
self.commitConfigUpdate('common-config',
'layouts/no-timer.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
# If APScheduler is in mid-event when we remove the job,
# we can end up with one more event firing, so give it an
@ -4101,7 +4101,7 @@ class TestScheduler(ZuulTestCase):
# the hold flag before the first job.
self.executor_server.hold_jobs_in_build = True
self.commitConfigUpdate('common-config', 'layouts/timer-smtp.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
# The pipeline triggers every second, so we should have seen
# several by now.
@ -4134,7 +4134,7 @@ class TestScheduler(ZuulTestCase):
# Stop queuing timer triggered jobs and let any that may have
# queued through so that end of test assertions pass.
self.commitConfigUpdate('common-config', 'layouts/no-timer.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
# If APScheduler is in mid-event when we remove the job, we
# can end up with one more event firing, so give it an extra
@ -4150,7 +4150,7 @@ class TestScheduler(ZuulTestCase):
self.worker.hold_jobs_in_build = True
self.config.set('zuul', 'layout_config',
'tests/fixtures/layout-timer.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.registerJobs()
# The pipeline triggers every second, so we should have seen
@ -4182,7 +4182,7 @@ class TestScheduler(ZuulTestCase):
# queued through so that end of test assertions pass.
self.config.set('zuul', 'layout_config',
'tests/fixtures/layout-no-timer.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.registerJobs()
self.waitUntilSettled()
# If APScheduler is in mid-event when we remove the job, we
@ -4491,7 +4491,7 @@ class TestScheduler(ZuulTestCase):
self.waitUntilSettled()
self.executor_server.hold_jobs_in_build = True
self.commitConfigUpdate('common-config', 'layouts/timer.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
# We expect that one build for each branch (master and stable) appears.
for _ in iterate_timeout(30, 'Wait for two builds that are hold'):
@ -4509,7 +4509,7 @@ class TestScheduler(ZuulTestCase):
self.commitConfigUpdate('common-config',
'layouts/no-timer.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
self.executor_server.hold_jobs_in_build = False
self.executor_server.release()
@ -5044,7 +5044,7 @@ class TestScheduler(ZuulTestCase):
self.commitConfigUpdate('org/common-config',
'layouts/rate-limit-reconfigure2.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
# D's remaining job should still be queued
@ -5075,7 +5075,7 @@ class TestScheduler(ZuulTestCase):
self.waitUntilSettled()
self.commitConfigUpdate('org/common-config',
'layouts/reconfigure-window2.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
tenant = self.sched.abide.tenants.get('tenant-one')
queue = tenant.layout.pipelines['gate'].queues[0]
# Even though we have configured a smaller window, the value
@ -5083,7 +5083,7 @@ class TestScheduler(ZuulTestCase):
self.assertEqual(queue.window, 20)
self.assertTrue(len(self.builds), 4)
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
tenant = self.sched.abide.tenants.get('tenant-one')
queue = tenant.layout.pipelines['gate'].queues[0]
self.assertEqual(queue.window, 20)
@ -5122,7 +5122,7 @@ class TestScheduler(ZuulTestCase):
self.waitUntilSettled()
self.commitConfigUpdate('org/common-config',
'layouts/reconfigure-window-fixed2.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
tenant = self.sched.abide.tenants.get('tenant-one')
queue = tenant.layout.pipelines['gate'].queues[0]
@ -5133,7 +5133,7 @@ class TestScheduler(ZuulTestCase):
# next pass through the queue processor.
self.assertEqual(len(self.builds), 4)
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
tenant = self.sched.abide.tenants.get('tenant-one')
queue = tenant.layout.pipelines['gate'].queues[0]
self.assertEqual(queue.window, 1)
@ -5181,7 +5181,7 @@ class TestScheduler(ZuulTestCase):
self.waitUntilSettled()
self.commitConfigUpdate('org/common-config',
'layouts/reconfigure-window-fixed2.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
self.log.debug("Reconfiguration complete")
@ -5196,7 +5196,7 @@ class TestScheduler(ZuulTestCase):
# run and marked B inactive; run another reconfiguration so
# that we're testing what happens when we reconfigure after
# the active window having shrunk.
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
# Unpause the node requests now
self.fake_nodepool.unpause()
@ -5237,13 +5237,13 @@ class TestScheduler(ZuulTestCase):
# Remove job2
self.commitConfigUpdate('org/common-config',
'layouts/reconfigure-remove-add2.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.assertTrue(len(self.builds), 1)
# Add job2 back
self.commitConfigUpdate('org/common-config',
'layouts/reconfigure-remove-add.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.assertTrue(len(self.builds), 2)
self.executor_server.hold_jobs_in_build = False
@ -5386,7 +5386,7 @@ For CI problems and help debugging, contact ci@example.org"""
to the correct reporter"""
self.updateConfigLayout(
'tests/fixtures/layout-merge-failure.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.registerJobs()
# Check a test failure isn't reported to SMTP
@ -5623,7 +5623,7 @@ For CI problems and help debugging, contact ci@example.org"""
# Now reload the configuration (simulate a HUP) to check the pipeline
# comes out of disabled
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
tenant = self.sched.abide.tenants.get('tenant-one')
@ -6056,7 +6056,7 @@ For CI problems and help debugging, contact ci@example.org"""
self.waitUntilSettled()
self.commitConfigUpdate('common-config', 'layouts/no-jobs.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
self.fake_nodepool.unpause()
@ -6822,7 +6822,7 @@ class TestSchedulerSuccessURL(ZuulTestCase):
def test_success_url(self):
"Ensure bad build params are ignored"
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.init_repo('org/docs')
A = self.fake_gerrit.addFakeChange('org/docs', 'master', 'A')
@ -7522,7 +7522,7 @@ class TestSemaphore(ZuulTestCase):
tenant.semaphore_handler.semaphores)
# reconfigure without layout change
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
tenant = self.sched.abide.tenants.get('tenant-one')
@ -7533,7 +7533,7 @@ class TestSemaphore(ZuulTestCase):
self.commitConfigUpdate(
'common-config',
'config/semaphore/zuul-reconfiguration.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
tenant = self.sched.abide.tenants.get('tenant-one')
@ -7565,7 +7565,7 @@ class TestSemaphore(ZuulTestCase):
self.commitConfigUpdate(
'common-config',
'config/semaphore/git/common-config/zuul-remove-job.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
# Release job project-test1 which should be the only job left
@ -7612,7 +7612,7 @@ class TestSemaphore(ZuulTestCase):
self.commitConfigUpdate(
'common-config',
'config/semaphore/git/common-config/zuul-remove-job.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
# Now we can unpause nodepool
@ -8095,7 +8095,7 @@ class TestSchedulerFailFast(ZuulTestCase):
# Commit new config that removes project-test1
self.commitConfigUpdate('common-config',
'layouts/fail-fast-reconfigure.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
# Release project-test1
self.executor_server.release('project-test1')
@ -8265,7 +8265,8 @@ class TestSchedulerSmartReconfiguration(ZuulTestCase):
self.newTenantConfig('config/multi-tenant/main-reconfig.yaml')
self.sched_app.smartReconfigure(command_socket=command_socket)
self.scheds.execute(
lambda app: app.smartReconfigure(command_socket=command_socket))
# Wait for smart reconfiguration. Only tenant-two should be
# reconfigured. Note that waitUntilSettled is not

View File

@ -384,7 +384,7 @@ class TestBranchDeletion(ZuulTestCase):
self.waitUntilSettled()
self.delete_branch('org/project', 'stable/queens')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
in_repo_conf = textwrap.dedent(
@ -999,7 +999,7 @@ class TestInRepoConfig(ZuulTestCase):
A.addApproval('Code-Review', 2)
self.fake_gerrit.addEvent(A.addApproval('Approved', 1))
self.waitUntilSettled()
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
gc.collect()
@ -3509,7 +3509,7 @@ class TestRoleBranches(RoleTestCase):
'parent-job-pre', 'parent-master-role')
self._addRole('project1', 'master', 'master-role', parent=p)
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
# Push a change to project2 which will run 3 jobs which
# inherit from project1.
self.executor_server.hold_jobs_in_build = True

View File

@ -1515,7 +1515,7 @@ class TestTenantScopedWebApi(BaseTestWeb):
self.create_branch('org/project', 'stable')
self.executor_server.hold_jobs_in_build = True
self.commitConfigUpdate('common-config', 'layouts/timer.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
for _ in iterate_timeout(30, 'Wait for a build on hold'):
@ -1548,7 +1548,7 @@ class TestTenantScopedWebApi(BaseTestWeb):
self.commitConfigUpdate('common-config',
'layouts/no-timer.yaml')
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
self.executor_server.hold_jobs_in_build = False
self.executor_server.release()

View File

@ -222,7 +222,7 @@ class TestZuulTriggerProjectChangeMerged(ZuulTestCase):
# check to make sure that we don't end up with a stale trigger
# cache that has references to projects from the old
# configuration.
self.sched.reconfigure(self.config)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
D.addApproval('Code-Review', 2)
self.fake_gerrit.addEvent(D.addApproval('Approved', 1))