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 random
import re import re
from logging import Logger from logging import Logger
from typing import Callable, Optional, Any, Iterable
import requests import requests
import select import select
@ -90,8 +91,7 @@ import zuul.configloader
from zuul.lib.config import get_default from zuul.lib.config import get_default
from zuul.lib.logutil import get_annotated_logger from zuul.lib.logutil import get_annotated_logger
FIXTURE_DIR = os.path.join(os.path.dirname(__file__), FIXTURE_DIR = os.path.join(os.path.dirname(__file__), 'fixtures')
'fixtures')
KEEP_TEMPDIRS = bool(os.environ.get('KEEP_TEMPDIRS', False)) KEEP_TEMPDIRS = bool(os.environ.get('KEEP_TEMPDIRS', False))
@ -3436,20 +3436,20 @@ class SchedulerTestApp:
self.sched.registerConnections(connections) self.sched.registerConnections(connections)
self.executor_client = zuul.executor.client.ExecutorClient( executor_client = zuul.executor.client.ExecutorClient(
self.config, self.sched) self.config, self.sched)
self.merge_client = RecordingMergeClient(self.config, self.sched) merge_client = RecordingMergeClient(self.config, self.sched)
self.nodepool = zuul.nodepool.Nodepool(self.sched) nodepool = zuul.nodepool.Nodepool(self.sched)
self.zk = zuul.zk.ZooKeeper(enable_cache=True) zk = zuul.zk.ZooKeeper(enable_cache=True)
self.zk.connect(self.zk_config, timeout=60.0) zk.connect(self.zk_config, timeout=30.0)
self.sched.setExecutor(self.executor_client) self.sched.setExecutor(executor_client)
self.sched.setMerger(self.merge_client) self.sched.setMerger(merge_client)
self.sched.setNodepool(self.nodepool) self.sched.setNodepool(nodepool)
self.sched.setZooKeeper(self.zk) self.sched.setZooKeeper(zk)
self.sched.start() self.sched.start()
self.executor_client.gearman.waitForServer() executor_client.gearman.waitForServer()
self.sched.reconfigure(self.config) self.sched.reconfigure(self.config)
self.sched.resume() self.sched.resume()
@ -3472,6 +3472,46 @@ class SchedulerTestApp:
self.log.exception("Reconfiguration failed:") 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): class ZuulTestCase(BaseTestCase):
"""A test case with a functioning Zuul. """A test case with a functioning Zuul.
@ -3660,21 +3700,21 @@ class ZuulTestCase(BaseTestCase):
self.history = self.executor_server.build_history self.history = self.executor_server.build_history
self.builds = self.executor_server.running_builds self.builds = self.executor_server.running_builds
self.sched_app = SchedulerTestApp(self.log, self.config, self.scheds = SchedulerTestManager()
self.zk_config, sched_app = self.scheds.create(
self.connections) self.log, self.config, self.zk_config, self.connections)
self.sched = self.sched_app.sched self.sched = sched_app.sched
self.event_queues = self.sched_app.event_queues + self.event_queues self.event_queues = sched_app.event_queues + self.event_queues
if hasattr(self, 'fake_github'): if hasattr(self, 'fake_github'):
self.event_queues.append( self.event_queues.append(
self.fake_github.github_event_connector._event_forward_queue) self.fake_github.github_event_connector._event_forward_queue)
self.executor_client = self.sched_app.executor_client self.executor_client = sched_app.sched.executor
self.merge_client = self.sched_app.merge_client self.merge_client = sched_app.sched.merger
self.merge_server = None self.merge_server = None
self.nodepool = self.sched_app.nodepool self.nodepool = sched_app.sched.nodepool
self.zk = self.sched_app.zk self.zk = sched_app.sched.zk
# Cleanups are run in reverse order # Cleanups are run in reverse order
self.addCleanup(self.assertCleanShutdown) self.addCleanup(self.assertCleanShutdown)
@ -4075,8 +4115,8 @@ class ZuulTestCase(BaseTestCase):
self.merge_server.join() self.merge_server.join()
self.executor_server.stop() self.executor_server.stop()
self.executor_server.join() self.executor_server.join()
self.sched.stop() self.scheds.execute(lambda app: app.sched.stop())
self.sched.join() self.scheds.execute(lambda app: app.sched.join())
self.statsd.stop() self.statsd.stop()
self.statsd.join() self.statsd.join()
self.rpcclient.shutdown() self.rpcclient.shutdown()
@ -4279,7 +4319,7 @@ class ZuulTestCase(BaseTestCase):
def areAllNodeRequestsComplete(self): def areAllNodeRequestsComplete(self):
if self.fake_nodepool.paused: if self.fake_nodepool.paused:
return True return True
if self.sched.nodepool.requests: if self.nodepool.requests:
return False return False
return True return True
@ -4338,7 +4378,8 @@ class ZuulTestCase(BaseTestCase):
# Join ensures that the queue is empty _and_ events have been # Join ensures that the queue is empty _and_ events have been
# processed # processed
self.eventQueuesJoin() self.eventQueuesJoin()
self.sched.run_handler_lock.acquire() self.scheds.execute(
lambda app: app.sched.run_handler_lock.acquire())
if (self.areAllMergeJobsWaiting() and if (self.areAllMergeJobsWaiting() and
self.haveAllBuildsReported() and self.haveAllBuildsReported() and
self.areAllBuildsWaiting() and self.areAllBuildsWaiting() and
@ -4349,14 +4390,16 @@ class ZuulTestCase(BaseTestCase):
# when locked the run handler and checked that the # when locked the run handler and checked that the
# components were stable, we don't erroneously # components were stable, we don't erroneously
# report that we are settled. # 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.executor_server.lock.release()
self.log.debug("...settled. (%s)", msg) self.log.debug("...settled. (%s)", msg)
self.logState() self.logState()
return return
self.sched.run_handler_lock.release() self.scheds.execute(
lambda app: app.sched.run_handler_lock.release())
self.executor_server.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): def waitForPoll(self, poller, timeout=30):
self.log.debug("Wait for poll on %s", poller) 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" "Test the SQL reporter fails gracefully when unable to connect"
self.config.set('zuul', 'layout_config', self.config.set('zuul', 'layout_config',
'tests/fixtures/layout-sql-reporter.yaml') '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 # Trigger a reporter. If no errors are raised, the reporter has been
# disabled correctly # disabled correctly

View File

@ -346,7 +346,7 @@ class TestGerritToGithubCRD(ZuulTestCase):
self.fake_gerrit.addEvent(A.getPatchsetCreatedEvent(1)) self.fake_gerrit.addEvent(A.getPatchsetCreatedEvent(1))
self.waitUntilSettled() 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 # Make sure the items still share a change queue, and the
# first one is not live. # first one is not live.
@ -795,7 +795,7 @@ class TestGithubToGerritCRD(ZuulTestCase):
self.fake_github.emitEvent(A.getPullRequestEditedEvent()) self.fake_github.emitEvent(A.getPullRequestEditedEvent())
self.waitUntilSettled() 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 # Make sure the items still share a change queue, and the
# first one is not live. # first one is not live.

View File

@ -281,7 +281,7 @@ class TestExecutorRepos(ZuulTestCase):
# Start timer trigger - also org/project # Start timer trigger - also org/project
self.commitConfigUpdate('common-config', self.commitConfigUpdate('common-config',
'layouts/repo-checkout-timer-override.yaml') '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 # The pipeline triggers every second, so we should have seen
# several by now. # several by now.
@ -292,7 +292,7 @@ class TestExecutorRepos(ZuulTestCase):
# below don't race against more jobs being queued. # below don't race against more jobs being queued.
self.commitConfigUpdate('common-config', self.commitConfigUpdate('common-config',
'layouts/repo-checkout-no-timer-override.yaml') 'layouts/repo-checkout-no-timer-override.yaml')
self.sched.reconfigure(self.config) self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled() self.waitUntilSettled()
# If APScheduler is in mid-event when we remove the job, we # 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 # 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 # Start timer trigger - also org/project
self.commitConfigUpdate('common-config', self.commitConfigUpdate('common-config',
'layouts/repo-checkout-timer.yaml') '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' p1 = 'review.example.com/org/project1'
projects = [p1] projects = [p1]
@ -339,7 +339,7 @@ class TestExecutorRepos(ZuulTestCase):
# below don't race against more jobs being queued. # below don't race against more jobs being queued.
self.commitConfigUpdate('common-config', self.commitConfigUpdate('common-config',
'layouts/repo-checkout-no-timer.yaml') 'layouts/repo-checkout-no-timer.yaml')
self.sched.reconfigure(self.config) self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled() self.waitUntilSettled()
# If APScheduler is in mid-event when we remove the job, we # 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 # 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', self.fake_gerrit.addFakeChecker(uuid='zuul_check:abcd',
repository='org/project', repository='org/project',
status='ENABLED') status='ENABLED')
self.sched.reconfigure(self.config) self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled() self.waitUntilSettled()
A = self.fake_gerrit.addFakeChange('org/project', 'master', 'A') 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.fake_gerrit.addEvent(A.getPatchsetCreatedEvent(1))
self.waitUntilSettled() 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 # Make sure the items still share a change queue, and the
# first one is not live. # first one is not live.

View File

@ -445,7 +445,7 @@ class TestGerritLegacyCRD(ZuulTestCase):
self.fake_gerrit.addEvent(A.getPatchsetCreatedEvent(1)) self.fake_gerrit.addEvent(A.getPatchsetCreatedEvent(1))
self.waitUntilSettled() 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 # Make sure the items still share a change queue, and the
# first one is not live. # first one is not live.

View File

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

View File

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

View File

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

View File

@ -1515,7 +1515,7 @@ class TestTenantScopedWebApi(BaseTestWeb):
self.create_branch('org/project', 'stable') self.create_branch('org/project', 'stable')
self.executor_server.hold_jobs_in_build = True self.executor_server.hold_jobs_in_build = True
self.commitConfigUpdate('common-config', 'layouts/timer.yaml') self.commitConfigUpdate('common-config', 'layouts/timer.yaml')
self.sched.reconfigure(self.config) self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled() self.waitUntilSettled()
for _ in iterate_timeout(30, 'Wait for a build on hold'): for _ in iterate_timeout(30, 'Wait for a build on hold'):
@ -1548,7 +1548,7 @@ class TestTenantScopedWebApi(BaseTestWeb):
self.commitConfigUpdate('common-config', self.commitConfigUpdate('common-config',
'layouts/no-timer.yaml') 'layouts/no-timer.yaml')
self.sched.reconfigure(self.config) self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled() self.waitUntilSettled()
self.executor_server.hold_jobs_in_build = False self.executor_server.hold_jobs_in_build = False
self.executor_server.release() 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 # check to make sure that we don't end up with a stale trigger
# cache that has references to projects from the old # cache that has references to projects from the old
# configuration. # configuration.
self.sched.reconfigure(self.config) self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
D.addApproval('Code-Review', 2) D.addApproval('Code-Review', 2)
self.fake_gerrit.addEvent(D.addApproval('Approved', 1)) self.fake_gerrit.addEvent(D.addApproval('Approved', 1))