summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--tests/unit/test_client.py158
-rwxr-xr-xzuul/cmd/client.py38
2 files changed, 108 insertions, 88 deletions
diff --git a/tests/unit/test_client.py b/tests/unit/test_client.py
index b51639952..f241147eb 100644
--- a/tests/unit/test_client.py
+++ b/tests/unit/test_client.py
@@ -27,10 +27,11 @@ import jwt
import testtools
from zuul.zk import ZooKeeperClient
+from zuul.zk.locks import SessionAwareLock
from zuul.cmd.client import parse_cutoff
from tests.base import BaseTestCase, ZuulTestCase
-from tests.base import FIXTURE_DIR
+from tests.base import FIXTURE_DIR, iterate_timeout
from kazoo.exceptions import NoNodeError
@@ -362,81 +363,112 @@ class TestOnlineZKOperations(ZuulTestCase):
def assertSQLState(self):
pass
- def test_delete_pipeline_check(self):
- self.executor_server.hold_jobs_in_build = True
- A = self.fake_gerrit.addFakeChange('org/project', 'master', 'A')
- self.fake_gerrit.addEvent(A.getPatchsetCreatedEvent(1))
- self.waitUntilSettled()
-
- config_file = os.path.join(self.test_root, 'zuul.conf')
- with open(config_file, 'w') as f:
- self.config.write(f)
-
- # Make sure the pipeline exists
- self.getZKTree('/zuul/tenant/tenant-one/pipeline/check/item')
- p = subprocess.Popen(
- [os.path.join(sys.prefix, 'bin/zuul-admin'),
- '-c', config_file,
- 'delete-pipeline-state',
- 'tenant-one', 'check',
- ],
- stdout=subprocess.PIPE)
- out, _ = p.communicate()
- self.log.debug(out.decode('utf8'))
- # Make sure it's deleted
- with testtools.ExpectedException(NoNodeError):
- self.getZKTree('/zuul/tenant/tenant-one/pipeline/check/item')
-
- self.executor_server.hold_jobs_in_build = False
- self.executor_server.release()
- B = self.fake_gerrit.addFakeChange('org/project', 'master', 'B')
- self.fake_gerrit.addEvent(B.getPatchsetCreatedEvent(1))
+ def _test_delete_pipeline(self, pipeline):
+ sched = self.scheds.first.sched
+ tenant = sched.abide.tenants['tenant-one']
+ # Force a reconfiguration due to a config change (so that the
+ # tenant trigger event queue gets a minimum timestamp set)
+ file_dict = {'zuul.yaml': ''}
+ M = self.fake_gerrit.addFakeChange('org/project', 'master', 'A',
+ files=file_dict)
+ M.setMerged()
+ self.fake_gerrit.addEvent(M.getChangeMergedEvent())
self.waitUntilSettled()
- self.assertHistory([
- dict(name='project-merge', result='SUCCESS', changes='1,1'),
- dict(name='project-merge', result='SUCCESS', changes='2,1'),
- dict(name='project-test1', result='SUCCESS', changes='2,1'),
- dict(name='project-test2', result='SUCCESS', changes='2,1'),
- ], ordered=False)
- def test_delete_pipeline_gate(self):
self.executor_server.hold_jobs_in_build = True
A = self.fake_gerrit.addFakeChange('org/project', 'master', 'A')
- A.addApproval('Code-Review', 2)
- self.fake_gerrit.addEvent(A.addApproval('Approved', 1))
+ if pipeline == 'check':
+ self.fake_gerrit.addEvent(A.getPatchsetCreatedEvent(1))
+ else:
+ A.addApproval('Code-Review', 2)
+ self.fake_gerrit.addEvent(A.addApproval('Approved', 1))
self.waitUntilSettled()
- config_file = os.path.join(self.test_root, 'zuul.conf')
- with open(config_file, 'w') as f:
- self.config.write(f)
-
- # Make sure the pipeline exists
- self.getZKTree('/zuul/tenant/tenant-one/pipeline/gate/item')
- p = subprocess.Popen(
- [os.path.join(sys.prefix, 'bin/zuul-admin'),
- '-c', config_file,
- 'delete-pipeline-state',
- 'tenant-one', 'gate',
- ],
- stdout=subprocess.PIPE)
- out, _ = p.communicate()
- self.log.debug(out.decode('utf8'))
- # Make sure it's deleted
- with testtools.ExpectedException(NoNodeError):
- self.getZKTree('/zuul/tenant/tenant-one/pipeline/gate/item')
+ # Lock the check pipeline so we don't process the event we're
+ # about to submit (it should go into the pipeline trigger event
+ # queue and stay there while we delete the pipeline state).
+ # This way we verify that events arrived before the deletion
+ # still work.
+ plock = SessionAwareLock(
+ self.zk_client.client,
+ f"/zuul/locks/pipeline/{tenant.name}/{pipeline}")
+ plock.acquire(blocking=True, timeout=None)
+ try:
+ self.log.debug('Got pipeline lock')
+ # Add a new event while our old last reconfigure time is
+ # in place.
+ B = self.fake_gerrit.addFakeChange('org/project', 'master', 'B')
+ if pipeline == 'check':
+ self.fake_gerrit.addEvent(B.getPatchsetCreatedEvent(1))
+ else:
+ B.addApproval('Code-Review', 2)
+ self.fake_gerrit.addEvent(B.addApproval('Approved', 1))
+
+ # Wait until it appears in the pipeline trigger event queue
+ self.log.debug('Waiting for event')
+ for x in iterate_timeout(30, 'trigger event queue has events'):
+ if sched.pipeline_trigger_events[
+ tenant.name][pipeline].hasEvents():
+ break
+ self.log.debug('Got event')
+ except Exception:
+ plock.release()
+ raise
+ # Grab the run handler lock so that we will continue to avoid
+ # further processing of the event after we release the
+ # pipeline lock (which the delete command needs to acquire).
+ sched.run_handler_lock.acquire()
+ try:
+ plock.release()
+ self.log.debug('Got run lock')
+ config_file = os.path.join(self.test_root, 'zuul.conf')
+ with open(config_file, 'w') as f:
+ self.config.write(f)
+
+ # Make sure the pipeline exists
+ self.getZKTree(
+ f'/zuul/tenant/{tenant.name}/pipeline/{pipeline}/item')
+ # Save the old layout uuid
+ tenant = sched.abide.tenants[tenant.name]
+ old_layout_uuid = tenant.layout.uuid
+ self.log.debug('Deleting pipeline state')
+ p = subprocess.Popen(
+ [os.path.join(sys.prefix, 'bin/zuul-admin'),
+ '-c', config_file,
+ 'delete-pipeline-state',
+ tenant.name, pipeline,
+ ],
+ stdout=subprocess.PIPE)
+ # Delete the pipeline state
+ out, _ = p.communicate()
+ self.log.debug(out.decode('utf8'))
+ # Make sure it's deleted
+ with testtools.ExpectedException(NoNodeError):
+ self.getZKTree(
+ f'/zuul/tenant/{tenant.name}/pipeline/{pipeline}/item')
+ finally:
+ sched.run_handler_lock.release()
self.executor_server.hold_jobs_in_build = False
self.executor_server.release()
- B = self.fake_gerrit.addFakeChange('org/project', 'master', 'B')
- B.addApproval('Code-Review', 2)
- self.fake_gerrit.addEvent(B.addApproval('Approved', 1))
self.waitUntilSettled()
self.assertHistory([
- dict(name='project-merge', result='SUCCESS', changes='1,1'),
dict(name='project-merge', result='SUCCESS', changes='2,1'),
- dict(name='project-test1', result='SUCCESS', changes='2,1'),
- dict(name='project-test2', result='SUCCESS', changes='2,1'),
+ dict(name='project-merge', result='SUCCESS', changes='3,1'),
+ dict(name='project-test1', result='SUCCESS', changes='3,1'),
+ dict(name='project-test2', result='SUCCESS', changes='3,1'),
], ordered=False)
+ tenant = sched.abide.tenants[tenant.name]
+ new_layout_uuid = tenant.layout.uuid
+ self.assertEqual(old_layout_uuid, new_layout_uuid)
+ self.assertEqual(tenant.layout.pipelines[pipeline].state.layout_uuid,
+ old_layout_uuid)
+
+ def test_delete_pipeline_check(self):
+ self._test_delete_pipeline('check')
+
+ def test_delete_pipeline_gate(self):
+ self._test_delete_pipeline('gate')
class TestDBPruneParse(BaseTestCase):
diff --git a/zuul/cmd/client.py b/zuul/cmd/client.py
index 031b10a1e..62e51ac3f 100755
--- a/zuul/cmd/client.py
+++ b/zuul/cmd/client.py
@@ -30,16 +30,14 @@ import time
import textwrap
import requests
import urllib.parse
-from uuid import uuid4
import zuul.cmd
from zuul.lib.config import get_default
-from zuul.model import SystemAttributes, PipelineState
+from zuul.model import SystemAttributes, PipelineState, PipelineChangeList
from zuul.zk import ZooKeeperClient
from zuul.lib.keystorage import KeyStorage
-from zuul.zk.locks import tenant_write_lock
+from zuul.zk.locks import tenant_read_lock, pipeline_lock
from zuul.zk.zkobject import ZKContext
-from zuul.zk.layout import LayoutState, LayoutStateStore
from zuul.zk.components import COMPONENT_REGISTRY
@@ -1029,28 +1027,18 @@ class Client(zuul.cmd.ZuulApp):
safe_tenant = urllib.parse.quote_plus(args.tenant)
safe_pipeline = urllib.parse.quote_plus(args.pipeline)
COMPONENT_REGISTRY.create(zk_client)
- with tenant_write_lock(zk_client, args.tenant) as lock:
+ self.log.info('get tenant')
+ with tenant_read_lock(zk_client, args.tenant):
path = f'/zuul/tenant/{safe_tenant}/pipeline/{safe_pipeline}'
- layout_uuid = None
- zk_client.client.delete(
- f'/zuul/tenant/{safe_tenant}/pipeline/{safe_pipeline}',
- recursive=True)
- with ZKContext(zk_client, lock, None, self.log) as context:
- ps = PipelineState.new(context, _path=path,
- layout_uuid=layout_uuid)
- # Force everyone to make a new layout for this tenant in
- # order to rebuild the shared change queues.
- layout_state = LayoutState(
- tenant_name=args.tenant,
- hostname='admin command',
- last_reconfigured=int(time.time()),
- last_reconfigure_event_ltime=-1,
- uuid=uuid4().hex,
- branch_cache_min_ltimes={},
- ltime=ps._zstat.last_modified_transaction_id,
- )
- tenant_layout_state = LayoutStateStore(zk_client, lambda: None)
- tenant_layout_state[args.tenant] = layout_state
+ self.log.info('get pipe')
+ with pipeline_lock(
+ zk_client, args.tenant, args.pipeline
+ ) as plock:
+ self.log.info('got locks')
+ zk_client.client.delete(path, recursive=True)
+ with ZKContext(zk_client, plock, None, self.log) as context:
+ PipelineState.new(context, _path=path, layout_uuid=None)
+ PipelineChangeList.new(context)
sys.exit(0)