Implement job freezing API in zuul-web
This implements the job freezing API directly in zuul-web, so there is no need to call the scheduler via RPC. Change-Id: Ibc7737a51fe5428bacdcb4763b3e6155cea29036
This commit is contained in:
@@ -57,7 +57,7 @@ class ExecutorClient(object):
|
||||
job, uuid, nodes, item.change, dependent_changes)
|
||||
|
||||
params = zuul.executor.common.construct_build_params(
|
||||
uuid, self.sched,
|
||||
uuid, self.sched.connections,
|
||||
job, item, pipeline, dependent_changes, merger_items,
|
||||
redact_secrets_and_keys=False)
|
||||
# TODO: deprecate and remove this variable?
|
||||
|
||||
@@ -17,7 +17,7 @@ import os
|
||||
from zuul.lib import strings
|
||||
|
||||
|
||||
def construct_build_params(uuid, sched, job, item, pipeline,
|
||||
def construct_build_params(uuid, connections, job, item, pipeline,
|
||||
dependent_changes=[], merger_items=[],
|
||||
redact_secrets_and_keys=True):
|
||||
"""Returns a list of all the parameters needed to build a job.
|
||||
@@ -160,7 +160,7 @@ def construct_build_params(uuid, sched, job, item, pipeline,
|
||||
except Exception:
|
||||
# We have to find the project this way because it may not
|
||||
# be registered in the tenant (ie, a foreign project).
|
||||
source = sched.connections.getSourceByCanonicalHostname(
|
||||
source = connections.getSourceByCanonicalHostname(
|
||||
change['project']['canonical_hostname'])
|
||||
project = source.getProject(change['project']['name'])
|
||||
|
||||
|
||||
@@ -19,14 +19,10 @@ import time
|
||||
from abc import ABCMeta
|
||||
from typing import List
|
||||
|
||||
import zuul.executor.common
|
||||
|
||||
from zuul import model
|
||||
from zuul.connection import BaseConnection
|
||||
from zuul.lib import encryption
|
||||
from zuul.lib.gearworker import ZuulGearWorker
|
||||
from zuul.lib.jsonutil import ZuulJSONEncoder
|
||||
from zuul.zk.zkobject import LocalZKContext
|
||||
|
||||
|
||||
class RPCListenerBase(metaclass=ABCMeta):
|
||||
@@ -157,8 +153,6 @@ class RPCListener(RPCListenerBase):
|
||||
'job_list',
|
||||
'project_get',
|
||||
'project_list',
|
||||
'project_freeze_jobs',
|
||||
'project_freeze_job',
|
||||
'pipeline_list',
|
||||
'key_get',
|
||||
'config_errors_list',
|
||||
@@ -427,71 +421,6 @@ class RPCListener(RPCListenerBase):
|
||||
job.sendWorkComplete(json.dumps(
|
||||
sorted(output, key=lambda project: project["name"])))
|
||||
|
||||
def handle_project_freeze_jobs(self, gear_job):
|
||||
args = json.loads(gear_job.arguments)
|
||||
tenant = self.sched.abide.tenants.get(args.get("tenant"))
|
||||
project = None
|
||||
pipeline = None
|
||||
if tenant:
|
||||
(trusted, project) = tenant.getProject(args.get("project"))
|
||||
pipeline = tenant.layout.pipelines.get(args.get("pipeline"))
|
||||
if not project or not pipeline:
|
||||
gear_job.sendWorkComplete(json.dumps(None))
|
||||
return
|
||||
|
||||
change = model.Branch(project)
|
||||
change.branch = args.get("branch", "master")
|
||||
context = LocalZKContext(self.log)
|
||||
queue = model.ChangeQueue.new(context, pipeline=pipeline)
|
||||
item = model.QueueItem.new(context, queue=queue, change=change,
|
||||
pipeline=queue.pipeline)
|
||||
item.freezeJobGraph(tenant.layout, context,
|
||||
skip_file_matcher=True,
|
||||
redact_secrets_and_keys=True)
|
||||
|
||||
output = []
|
||||
|
||||
for job in item.current_build_set.job_graph.getJobs():
|
||||
output.append({
|
||||
'name': job.name,
|
||||
'dependencies':
|
||||
list(map(lambda x: x.toDict(), job.dependencies)),
|
||||
})
|
||||
|
||||
gear_job.sendWorkComplete(json.dumps(output))
|
||||
|
||||
def handle_project_freeze_job(self, gear_job):
|
||||
args = json.loads(gear_job.arguments)
|
||||
tenant = self.sched.abide.tenants.get(args.get("tenant"))
|
||||
project = None
|
||||
pipeline = None
|
||||
if tenant:
|
||||
(trusted, project) = tenant.getProject(args.get("project"))
|
||||
pipeline = tenant.layout.pipelines.get(args.get("pipeline"))
|
||||
if not project or not pipeline:
|
||||
gear_job.sendWorkComplete(json.dumps(None))
|
||||
return
|
||||
|
||||
change = model.Branch(project)
|
||||
change.branch = args.get("branch", "master")
|
||||
context = LocalZKContext(self.log)
|
||||
queue = model.ChangeQueue.new(context, pipeline=pipeline)
|
||||
item = model.QueueItem.new(context, queue=queue, change=change,
|
||||
pipeline=queue.pipeline)
|
||||
item.freezeJobGraph(tenant.layout, context,
|
||||
skip_file_matcher=True,
|
||||
redact_secrets_and_keys=True)
|
||||
|
||||
job = item.current_build_set.jobs.get(args.get("job"))
|
||||
if not job:
|
||||
gear_job.sendWorkComplete(json.dumps(None))
|
||||
return
|
||||
uuid = '0' * 32
|
||||
params = zuul.executor.common.construct_build_params(
|
||||
uuid, self.sched, job, item, pipeline)
|
||||
params['zuul']['buildset'] = None
|
||||
gear_job.sendWorkComplete(json.dumps(params, cls=ZuulJSONEncoder))
|
||||
|
||||
def handle_allowed_labels_get(self, job):
|
||||
args = json.loads(job.arguments)
|
||||
tenant = self.sched.abide.tenants.get(args.get("tenant"))
|
||||
|
||||
@@ -31,6 +31,7 @@ import select
|
||||
import ssl
|
||||
import threading
|
||||
|
||||
import zuul.executor.common
|
||||
from zuul import exceptions
|
||||
from zuul.configloader import ConfigLoader
|
||||
import zuul.lib.repl
|
||||
@@ -39,7 +40,15 @@ from zuul.lib import streamer_utils
|
||||
from zuul.lib.ansible import AnsibleManager
|
||||
from zuul.lib.keystorage import KeyStorage
|
||||
from zuul.lib.re2util import filter_allowed_disallowed
|
||||
from zuul.model import Abide, SystemAttributes, UnparsedAbideConfig, WebInfo
|
||||
from zuul.model import (
|
||||
Abide,
|
||||
Branch,
|
||||
ChangeQueue,
|
||||
QueueItem,
|
||||
SystemAttributes,
|
||||
UnparsedAbideConfig,
|
||||
WebInfo,
|
||||
)
|
||||
import zuul.rpcclient
|
||||
from zuul.version import get_version_string
|
||||
from zuul.zk import ZooKeeperClient
|
||||
@@ -50,6 +59,7 @@ from zuul.zk.layout import LayoutStateStore
|
||||
from zuul.zk.locks import tenant_read_lock
|
||||
from zuul.zk.nodepool import ZooKeeperNodepool
|
||||
from zuul.zk.system import ZuulSystem
|
||||
from zuul.zk.zkobject import LocalZKContext
|
||||
from zuul.lib.auth import AuthenticatorRegistry
|
||||
from zuul.lib.config import get_default
|
||||
|
||||
@@ -1178,19 +1188,20 @@ class ZuulWebAPI(object):
|
||||
@cherrypy.expose
|
||||
@cherrypy.tools.save_params()
|
||||
@cherrypy.tools.json_out(content_type='application/json; charset=utf-8')
|
||||
def project_freeze_jobs(self, tenant, pipeline, project, branch):
|
||||
job = self.rpc.submitJob(
|
||||
'zuul:project_freeze_jobs',
|
||||
{
|
||||
'tenant': tenant,
|
||||
'project': project,
|
||||
'pipeline': pipeline,
|
||||
'branch': branch
|
||||
}
|
||||
)
|
||||
ret = json.loads(job.data[0])
|
||||
if not ret:
|
||||
raise cherrypy.HTTPError(404)
|
||||
def project_freeze_jobs(self, tenant_name, pipeline_name, project_name,
|
||||
branch_name):
|
||||
item = self._freeze_jobs(
|
||||
tenant_name, pipeline_name, project_name, branch_name)
|
||||
|
||||
output = []
|
||||
for job in item.current_build_set.job_graph.getJobs():
|
||||
output.append({
|
||||
'name': job.name,
|
||||
'dependencies':
|
||||
list(map(lambda x: x.toDict(), job.dependencies)),
|
||||
})
|
||||
|
||||
ret = output
|
||||
resp = cherrypy.response
|
||||
resp.headers['Access-Control-Allow-Origin'] = '*'
|
||||
return ret
|
||||
@@ -1198,27 +1209,51 @@ class ZuulWebAPI(object):
|
||||
@cherrypy.expose
|
||||
@cherrypy.tools.save_params()
|
||||
@cherrypy.tools.json_out(content_type='application/json; charset=utf-8')
|
||||
def project_freeze_job(self, tenant, pipeline, project, branch, job):
|
||||
def project_freeze_job(self, tenant_name, pipeline_name, project_name,
|
||||
branch_name, job_name):
|
||||
# TODO(jhesketh): Allow a canonical change/item to be passed in which
|
||||
# would return the job with any in-change modifications.
|
||||
|
||||
job = self.rpc.submitJob(
|
||||
'zuul:project_freeze_job',
|
||||
{
|
||||
'tenant': tenant,
|
||||
'project': project,
|
||||
'pipeline': pipeline,
|
||||
'branch': branch,
|
||||
'job': job
|
||||
}
|
||||
)
|
||||
ret = json.loads(job.data[0])
|
||||
if not ret:
|
||||
item = self._freeze_jobs(
|
||||
tenant_name, pipeline_name, project_name, branch_name)
|
||||
job = item.current_build_set.jobs.get(job_name)
|
||||
if not job:
|
||||
raise cherrypy.HTTPError(404)
|
||||
|
||||
uuid = "0" * 32
|
||||
params = zuul.executor.common.construct_build_params(
|
||||
uuid, self.zuulweb.connections, job, item, item.pipeline)
|
||||
params['zuul']['buildset'] = None
|
||||
|
||||
ret = params
|
||||
resp = cherrypy.response
|
||||
resp.headers['Access-Control-Allow-Origin'] = '*'
|
||||
return ret
|
||||
|
||||
def _freeze_jobs(self, tenant_name, pipeline_name, project_name,
|
||||
branch_name):
|
||||
|
||||
tenant = self.zuulweb.abide.tenants.get(tenant_name)
|
||||
project = None
|
||||
pipeline = None
|
||||
|
||||
if tenant:
|
||||
_, project = tenant.getProject(project_name)
|
||||
pipeline = tenant.layout.pipelines.get(pipeline_name)
|
||||
if not project or not pipeline:
|
||||
raise cherrypy.HTTPError(404)
|
||||
|
||||
change = Branch(project)
|
||||
change.branch = branch_name or "master"
|
||||
context = LocalZKContext(self.log)
|
||||
queue = ChangeQueue.new(context, pipeline=pipeline)
|
||||
item = QueueItem.new(context, queue=queue, change=change,
|
||||
pipeline=queue.pipeline)
|
||||
item.freezeJobGraph(tenant.layout, context,
|
||||
skip_file_matcher=True,
|
||||
redact_secrets_and_keys=True)
|
||||
|
||||
return item
|
||||
|
||||
|
||||
class StaticHandler(object):
|
||||
def __init__(self, root):
|
||||
@@ -1449,14 +1484,15 @@ class ZuulWeb(object):
|
||||
controller=api, action='project')
|
||||
route_map.connect(
|
||||
'api',
|
||||
'/api/tenant/{tenant}/pipeline/{pipeline}'
|
||||
'/project/{project:.*}/branch/{branch:.*}/freeze-jobs',
|
||||
'/api/tenant/{tenant_name}/pipeline/{pipeline_name}'
|
||||
'/project/{project_name:.*}/branch/{branch_name:.*}/freeze-jobs',
|
||||
controller=api, action='project_freeze_jobs'
|
||||
)
|
||||
route_map.connect(
|
||||
'api',
|
||||
'/api/tenant/{tenant}/pipeline/{pipeline}'
|
||||
'/project/{project:.*}/branch/{branch:.*}/freeze-job/{job}',
|
||||
'/api/tenant/{tenant_name}/pipeline/{pipeline_name}'
|
||||
'/project/{project_name:.*}/branch/{branch_name:.*}'
|
||||
'/freeze-job/{job_name}',
|
||||
controller=api, action='project_freeze_job'
|
||||
)
|
||||
route_map.connect('api', '/api/tenant/{tenant}/pipelines',
|
||||
|
||||
Reference in New Issue
Block a user