Support GitHub PR webhooks

Story: 2000774

Change-Id: I2713c5d19326213539689e9d822831a393b2bf19
Co-Authored-By: Wayne Warren <waynr+launchpad@sdf.org>
Co-Authored-By: Jan Hruban <jan.hruban@gooddata.com>
Co-Authored-By: Jesse Keating <omgjlk@us.ibm.com>
This commit is contained in:
Gregory Haynes 2015-04-22 20:38:06 -07:00 committed by Jesse Keating
parent 2a53567014
commit 4fc1254907
14 changed files with 754 additions and 14 deletions

View File

@ -65,6 +65,24 @@ want Zuul to gate. For instance, you may want to grant ``Verified
be added to Gerrit. Zuul is very flexible and can take advantage of be added to Gerrit. Zuul is very flexible and can take advantage of
those. those.
GitHub
------
Create a connection with GitHub.
**driver=github**
**api_token**
API token for accessing GitHub.
See `Creating an access token for command-line use
<https://help.github.com/articles/creating-an-access-token-for-command-line-use/>`_.
**webhook_token**
Optional: Token for validating the webhook event payloads.
If not specified, payloads are not validated.
See `Securing your webhooks
<https://developer.github.com/webhooks/securing/>`_.
SMTP SMTP
---- ----

View File

@ -4,7 +4,7 @@ Triggers
======== ========
The process of merging a change starts with proposing a change to be The process of merging a change starts with proposing a change to be
merged. Primarily, Zuul supports Gerrit as a triggering system. merged. Zuul supports Gerrit and GitHub as triggering systems.
Zuul's design is modular, so alternate triggering and reporting Zuul's design is modular, so alternate triggering and reporting
systems can be supported. systems can be supported.
@ -100,6 +100,38 @@ the following options.
*require-approval* but will fail to enter the pipeline if there is *require-approval* but will fail to enter the pipeline if there is
a matching approval. a matching approval.
GitHub
------
Github webhook events can be configured as triggers.
A connection name with the github driver can take multiple events with the
following options.
**event**
The pull request event from github. A ``pull_request`` event will
have associated action(s) to trigger from. The supported actions are:
*opened* - pull request opened
*changed* - pull request synchronized
*closed* - pull request closed
*reopened* - pull request reopened
GitHub Configuration
~~~~~~~~~~~~~~~~~~~~
Configure GitHub `webhook events
<https://developer.github.com/webhooks/creating/>`_.
Set *Payload URL* to
``http://<zuul-hostname>/connection/<connection-name>/payload``.
Set *Content Type* to ``application/json``.
Select *Events* you are interested in. See above for the supported events.
Timer Timer
----- -----
@ -154,4 +186,4 @@ can be used by listing **zuul** as the trigger.
*reject-approval* *reject-approval*
This takes a list of approvals in the same format as This takes a list of approvals in the same format as
*require-approval* but will fail to enter the pipeline if there is *require-approval* but will fail to enter the pipeline if there is
a matching approval. a matching approval.

View File

@ -1,5 +1,6 @@
pbr>=1.1.0 pbr>=1.1.0
Github3.py
PyYAML>=3.1.0 PyYAML>=3.1.0
Paste Paste
WebOb>=1.2.3 WebOb>=1.2.3

View File

@ -58,6 +58,7 @@ import yaml
import zuul.driver.gerrit.gerritsource as gerritsource import zuul.driver.gerrit.gerritsource as gerritsource
import zuul.driver.gerrit.gerritconnection as gerritconnection import zuul.driver.gerrit.gerritconnection as gerritconnection
import zuul.driver.github.githubconnection as githubconnection
import zuul.scheduler import zuul.scheduler
import zuul.webapp import zuul.webapp
import zuul.rpclistener import zuul.rpclistener
@ -126,12 +127,12 @@ def simple_layout(path, driver='gerrit'):
return decorator return decorator
class ChangeReference(git.Reference): class GerritChangeReference(git.Reference):
_common_path_default = "refs/changes" _common_path_default = "refs/changes"
_points_to_commits_only = True _points_to_commits_only = True
class FakeChange(object): class FakeGerritChange(object):
categories = {'approved': ('Approved', -1, 1), categories = {'approved': ('Approved', -1, 1),
'code-review': ('Code-Review', -2, 2), 'code-review': ('Code-Review', -2, 2),
'verified': ('Verified', -2, 2)} 'verified': ('Verified', -2, 2)}
@ -139,6 +140,7 @@ class FakeChange(object):
def __init__(self, gerrit, number, project, branch, subject, def __init__(self, gerrit, number, project, branch, subject,
status='NEW', upstream_root=None, files={}): status='NEW', upstream_root=None, files={}):
self.gerrit = gerrit self.gerrit = gerrit
self.source = gerrit
self.reported = 0 self.reported = 0
self.queried = 0 self.queried = 0
self.patchsets = [] self.patchsets = []
@ -178,9 +180,9 @@ class FakeChange(object):
def addFakeChangeToRepo(self, msg, files, large): def addFakeChangeToRepo(self, msg, files, large):
path = os.path.join(self.upstream_root, self.project) path = os.path.join(self.upstream_root, self.project)
repo = git.Repo(path) repo = git.Repo(path)
ref = ChangeReference.create(repo, '1/%s/%s' % (self.number, ref = GerritChangeReference.create(
self.latest_patchset), repo, '1/%s/%s' % (self.number, self.latest_patchset),
'refs/tags/init') 'refs/tags/init')
repo.head.reference = ref repo.head.reference = ref
zuul.merger.merger.reset_repo_to_head(repo) zuul.merger.merger.reset_repo_to_head(repo)
repo.git.clean('-x', '-f', '-d') repo.git.clean('-x', '-f', '-d')
@ -469,9 +471,9 @@ class FakeGerritConnection(gerritconnection.GerritConnection):
files=None): files=None):
"""Add a change to the fake Gerrit.""" """Add a change to the fake Gerrit."""
self.change_number += 1 self.change_number += 1
c = FakeChange(self, self.change_number, project, branch, subject, c = FakeGerritChange(self, self.change_number, project, branch,
upstream_root=self.upstream_root, subject, upstream_root=self.upstream_root,
status=status, files=files) status=status, files=files)
self.changes[self.change_number] = c self.changes[self.change_number] = c
return c return c
@ -536,6 +538,162 @@ class FakeGerritConnection(gerritconnection.GerritConnection):
return os.path.join(self.upstream_root, project.name) return os.path.join(self.upstream_root, project.name)
class GithubChangeReference(git.Reference):
_common_path_default = "refs/pull"
_points_to_commits_only = True
class FakeGithubPullRequest(object):
def __init__(self, github, number, project, branch,
upstream_root, number_of_commits=1):
"""Creates a new PR with several commits.
Sends an event about opened PR."""
self.github = github
self.source = github
self.number = number
self.project = project
self.branch = branch
self.upstream_root = upstream_root
self.comments = []
self.updated_at = None
self.head_sha = None
self._createPRRef()
self._addCommitToRepo()
self._updateTimeStamp()
def addCommit(self):
"""Adds a commit on top of the actual PR head."""
self._addCommitToRepo()
self._updateTimeStamp()
def forcePush(self):
"""Clears actual commits and add a commit on top of the base."""
self._addCommitToRepo(reset=True)
self._updateTimeStamp()
def getPullRequestOpenedEvent(self):
return self._getPullRequestEvent('opened')
def getPullRequestSynchronizeEvent(self):
return self._getPullRequestEvent('synchronize')
def getPullRequestReopenedEvent(self):
return self._getPullRequestEvent('reopened')
def getPullRequestClosedEvent(self):
return self._getPullRequestEvent('closed')
def addComment(self, message):
self.comments.append(message)
self._updateTimeStamp()
def _getRepo(self):
repo_path = os.path.join(self.upstream_root, self.project)
return git.Repo(repo_path)
def _createPRRef(self):
repo = self._getRepo()
GithubChangeReference.create(
repo, self._getPRReference(), 'refs/tags/init')
def _addCommitToRepo(self, reset=False):
repo = self._getRepo()
ref = repo.references[self._getPRReference()]
if reset:
ref.set_object('refs/tags/init')
repo.head.reference = ref
zuul.merger.merger.reset_repo_to_head(repo)
repo.git.clean('-x', '-f', '-d')
fn = '%s-%s' % (self.branch.replace('/', '_'), self.number)
msg = 'test-%s' % self.number
fn = os.path.join(repo.working_dir, fn)
f = open(fn, 'w')
with open(fn, 'w') as f:
f.write("test %s %s\n" %
(self.branch, self.number))
repo.index.add([fn])
self.head_sha = repo.index.commit(msg).hexsha
repo.head.reference = 'master'
zuul.merger.merger.reset_repo_to_head(repo)
repo.git.clean('-x', '-f', '-d')
repo.heads['master'].checkout()
def _updateTimeStamp(self):
self.updated_at = time.strftime('%Y-%m-%dT%H:%M:%SZ', time.localtime())
def getPRHeadSha(self):
repo = self._getRepo()
return repo.references[self._getPRReference()].commit.hexsha
def _getPRReference(self):
return '%s/head' % self.number
def _getPullRequestEvent(self, action):
name = 'pull_request'
data = {
'action': action,
'number': self.number,
'pull_request': {
'number': self.number,
'updated_at': self.updated_at,
'base': {
'ref': self.branch,
'repo': {
'full_name': self.project
}
},
'head': {
'sha': self.head_sha
}
}
}
return (name, data)
class FakeGithubConnection(githubconnection.GithubConnection):
log = logging.getLogger("zuul.test.FakeGithubConnection")
def __init__(self, driver, connection_name, connection_config,
upstream_root=None):
super(FakeGithubConnection, self).__init__(driver, connection_name,
connection_config)
self.connection_name = connection_name
self.pr_number = 0
self.pull_requests = []
self.upstream_root = upstream_root
def openFakePullRequest(self, project, branch):
self.pr_number += 1
pull_request = FakeGithubPullRequest(
self, self.pr_number, project, branch, self.upstream_root)
self.pull_requests.append(pull_request)
return pull_request
def emitEvent(self, event):
"""Emulates sending the GitHub webhook event to the connection."""
port = self.webapp.server.socket.getsockname()[1]
name, data = event
payload = json.dumps(data)
headers = {'X-Github-Event': name}
req = urllib.request.Request(
'http://localhost:%s/connection/%s/payload'
% (port, self.connection_name),
data=payload, headers=headers)
urllib.request.urlopen(req)
def getGitUrl(self, project):
return os.path.join(self.upstream_root, str(project))
def getProjectBranches(self, project):
"""Masks getProjectBranches since we don't have a real github"""
# just returns master for now
return ['master']
class BuildHistory(object): class BuildHistory(object):
def __init__(self, **kw): def __init__(self, **kw):
self.__dict__.update(kw) self.__dict__.update(kw)
@ -701,7 +859,7 @@ class FakeBuild(object):
""" """
for change in changes: for change in changes:
hostname = change.gerrit.canonical_hostname hostname = change.source.canonical_hostname
path = os.path.join(self.jobdir.src_root, hostname, change.project) path = os.path.join(self.jobdir.src_root, hostname, change.project)
try: try:
repo = git.Repo(path) repo = git.Repo(path)
@ -1451,6 +1609,16 @@ class ZuulTestCase(BaseTestCase):
'zuul.driver.gerrit.GerritDriver.getConnection', 'zuul.driver.gerrit.GerritDriver.getConnection',
getGerritConnection)) getGerritConnection))
def getGithubConnection(driver, name, config):
con = FakeGithubConnection(driver, name, config,
upstream_root=self.upstream_root)
setattr(self, 'fake_' + name, con)
return con
self.useFixture(fixtures.MonkeyPatch(
'zuul.driver.github.GithubDriver.getConnection',
getGithubConnection))
# Set up smtp related fakes # Set up smtp related fakes
# TODO(jhesketh): This should come from lib.connections for better # TODO(jhesketh): This should come from lib.connections for better
# coverage # coverage

View File

@ -0,0 +1,22 @@
- pipeline:
name: check
manager: independent
trigger:
github:
- event: pull_request
action:
- opened
- changed
- reopened
- job:
name: project-test1
- job:
name: project-test2
- project:
name: org/project
check:
jobs:
- project-test1
- project-test2

17
tests/fixtures/zuul-github-driver.conf vendored Normal file
View File

@ -0,0 +1,17 @@
[gearman]
server=127.0.0.1
[zuul]
job_name_in_report=true
[merger]
git_dir=/tmp/zuul-test/git
git_user_email=zuul@example.com
git_user_name=zuul
zuul_url=http://zuul.example.com/p
[executor]
git_dir=/tmp/zuul-test/executor-git
[connection github]
driver=github

View File

@ -0,0 +1,52 @@
# Copyright 2015 GoodData
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
import logging
from tests.base import ZuulTestCase, simple_layout
logging.basicConfig(level=logging.DEBUG,
format='%(asctime)s %(name)-32s '
'%(levelname)-8s %(message)s')
class TestGithubDriver(ZuulTestCase):
config_file = 'zuul-github-driver.conf'
@simple_layout('layouts/basic-github.yaml', driver='github')
def test_pull_event(self):
self.executor_server.hold_jobs_in_build = True
pr = self.fake_github.openFakePullRequest('org/project', 'master')
self.fake_github.emitEvent(pr.getPullRequestOpenedEvent())
self.waitUntilSettled()
build_params = self.builds[0].parameters
self.assertEqual('master', build_params['ZUUL_BRANCH'])
self.assertEqual(str(pr.number), build_params['ZUUL_CHANGE'])
self.assertEqual(pr.head_sha, build_params['ZUUL_PATCHSET'])
self.executor_server.hold_jobs_in_build = False
self.executor_server.release()
self.waitUntilSettled()
self.assertEqual('SUCCESS',
self.getJobFromHistory('project-test1').result)
self.assertEqual('SUCCESS',
self.getJobFromHistory('project-test2').result)
job = self.getJobFromHistory('project-test2')
zuulvars = job.parameters['vars']['zuul']
self.assertEqual(pr.number, zuulvars['change'])
self.assertEqual(pr.head_sha, zuulvars['patchset'])

View File

@ -0,0 +1,36 @@
# Copyright 2017 IBM Corp.
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
from zuul.driver import Driver, ConnectionInterface, TriggerInterface
from zuul.driver import SourceInterface
import githubconnection
import githubtrigger
import githubsource
class GithubDriver(Driver, ConnectionInterface, TriggerInterface,
SourceInterface):
name = 'github'
def getConnection(self, name, config):
return githubconnection.GithubConnection(self, name, config)
def getTrigger(self, connection, config=None):
return githubtrigger.GithubTrigger(self, connection, config)
def getSource(self, connection):
return githubsource.GithubSource(self, connection)
def getTriggerSchema(self):
return githubtrigger.getSchema()

View File

@ -0,0 +1,228 @@
# Copyright 2015 Hewlett-Packard Development Company, L.P.
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
import logging
import hmac
import hashlib
import time
import webob
import webob.dec
import voluptuous as v
import github3
from zuul.connection import BaseConnection
from zuul.model import PullRequest, Ref, TriggerEvent
class GithubWebhookListener():
log = logging.getLogger("zuul.GithubWebhookListener")
def __init__(self, connection):
self.connection = connection
def handle_request(self, path, tenant_name, request):
if request.method != 'POST':
self.log.debug("Only POST method is allowed.")
raise webob.exc.HTTPMethodNotAllowed(
'Only POST method is allowed.')
self.log.debug("Github Webhook Received.")
self._validate_signature(request)
self.__dispatch_event(request)
def __dispatch_event(self, request):
try:
event = request.headers['X-Github-Event']
self.log.debug("X-Github-Event: " + event)
except KeyError:
self.log.debug("Request headers missing the X-Github-Event.")
raise webob.exc.HTTPBadRequest('Please specify a X-Github-Event '
'header.')
try:
method = getattr(self, '_event_' + event)
except AttributeError:
message = "Unhandled X-Github-Event: {0}".format(event)
self.log.debug(message)
raise webob.exc.HTTPBadRequest(message)
try:
event = method(request)
except:
self.log.exception('Exception when handling event:')
if event:
event.project_hostname = self.connection.canonical_hostname
self.log.debug('Scheduling github event: {0}'.format(event.type))
self.connection.sched.addEvent(event)
def _event_pull_request(self, request):
body = request.json_body
action = body.get('action')
pr_body = body.get('pull_request')
event = self._pull_request_to_event(pr_body)
event.type = 'pull_request'
if action == 'opened':
event.action = 'opened'
elif action == 'synchronize':
event.action = 'changed'
elif action == 'closed':
event.action = 'closed'
elif action == 'reopened':
event.action = 'reopened'
else:
return None
return event
def _validate_signature(self, request):
secret = self.connection.connection_config.get('webhook_token', None)
if secret is None:
return True
body = request.body
try:
request_signature = request.headers['X-Hub-Signature']
except KeyError:
raise webob.exc.HTTPUnauthorized(
'Please specify a X-Hub-Signature header with secret.')
payload_signature = 'sha1=' + hmac.new(secret,
body,
hashlib.sha1).hexdigest()
self.log.debug("Payload Signature: {0}".format(str(payload_signature)))
self.log.debug("Request Signature: {0}".format(str(request_signature)))
if str(payload_signature) != str(request_signature):
raise webob.exc.HTTPUnauthorized(
'Request signature does not match calculated payload '
'signature. Check that secret is correct.')
return True
def _pull_request_to_event(self, pr_body):
event = TriggerEvent()
event.trigger_name = 'github'
base = pr_body.get('base')
base_repo = base.get('repo')
head = pr_body.get('head')
event.project_name = base_repo.get('full_name')
event.change_number = pr_body.get('number')
event.change_url = self.connection.getPullUrl(event.project_name,
event.change_number)
event.updated_at = pr_body.get('updated_at')
event.branch = base.get('ref')
event.refspec = "refs/pull/" + str(pr_body.get('number')) + "/head"
event.patch_number = head.get('sha')
return event
class GithubConnection(BaseConnection):
driver_name = 'github'
log = logging.getLogger("connection.github")
payload_path = 'payload'
def __init__(self, driver, connection_name, connection_config):
super(GithubConnection, self).__init__(
driver, connection_name, connection_config)
self.github = None
self.canonical_hostname = self.connection_config.get(
'canonical_hostname', 'github.com')
self._change_cache = {}
self.projects = {}
self.source = driver.getSource(self)
def onLoad(self):
webhook_listener = GithubWebhookListener(self)
self.registerHttpHandler(self.payload_path,
webhook_listener.handle_request)
self._authenticateGithubAPI()
def onStop(self):
self.unregisterHttpHandler(self.payload_path)
def _authenticateGithubAPI(self):
token = self.connection_config.get('api_token', None)
if token is not None:
self.github = github3.login(token)
self.log.info("Github API Authentication successful.")
else:
self.github = None
self.log.info(
"No Github credentials found in zuul configuration, cannot "
"authenticate.")
def maintainCache(self, relevant):
for key, change in self._change_cache.items():
if change not in relevant:
del self._change_cache[key]
def getChange(self, event):
"""Get the change representing an event."""
if event.change_number:
change = PullRequest(event.project_name)
change.project = self.source.getProject(event.project_name)
change.number = event.change_number
change.refspec = event.refspec
change.branch = event.branch
change.url = event.change_url
change.updated_at = self._ghTimestampToDate(event.updated_at)
change.patchset = event.patch_number
else:
project = self.source.getProject(event.project_name)
change = Ref(project)
return change
def getGitUrl(self, project):
url = 'https://%s/%s' % ("github.com", project)
return url
def getGitwebUrl(self, project, sha=None):
url = 'https://%s/%s' % ("github.com", project)
if sha is not None:
url += '/commit/%s' % sha
return url
def getProject(self, name):
return self.projects.get(name)
def addProject(self, project):
self.projects[project.name] = project
def getProjectBranches(self, project):
owner, proj = project.name.split('/')
repository = self.github.repository(owner, proj)
branches = [branch.name for branch in repository.branches()]
return branches
def getPullUrl(self, project, number):
return '%s/pull/%s' % (self.getGitwebUrl(project), number)
def _ghTimestampToDate(self, timestamp):
return time.strptime(timestamp, '%Y-%m-%dT%H:%M:%SZ')
def getSchema():
github_connection = v.Any(str, v.Schema({}, extra=True))
return github_connection

View File

@ -0,0 +1,81 @@
# Copyright 2014 Puppet Labs Inc
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
import logging
import time
from zuul.source import BaseSource
from zuul.model import Project
class GithubSource(BaseSource):
name = 'github'
log = logging.getLogger("zuul.source.GithubSource")
def __init__(self, driver, connection, config=None):
hostname = connection.canonical_hostname
super(GithubSource, self).__init__(driver, connection,
hostname, config)
def getRefSha(self, project, ref):
"""Return a sha for a given project ref."""
raise NotImplementedError()
def waitForRefSha(self, project, ref, old_sha=''):
"""Block until a ref shows up in a given project."""
raise NotImplementedError()
def isMerged(self, change, head=None):
"""Determine if change is merged."""
raise NotImplementedError()
def canMerge(self, change, allow_needs):
"""Determine if change can merge."""
raise NotImplementedError()
def postConfig(self):
"""Called after configuration has been processed."""
pass
def getChange(self, event):
return self.connection.getChange(event)
def getProject(self, name):
p = self.connection.getProject(name)
if not p:
p = Project(name, self)
self.connection.addProject(p)
return p
def getProjectBranches(self, project):
return self.connection.getProjectBranches(project)
def getProjectOpenChanges(self, project):
"""Get the open changes for a project."""
raise NotImplementedError()
def updateChange(self, change, history=None):
"""Update information for a change."""
raise NotImplementedError()
def getGitUrl(self, project):
"""Get the git url for a project."""
return self.connection.getGitUrl(project)
def getGitwebUrl(self, project, sha=None):
"""Get the git-web url for a project."""
raise NotImplementedError()
def _ghTimestampToDate(self, timestamp):
return time.strptime(timestamp, '%Y-%m-%dT%H:%M:%SZ')

View File

@ -0,0 +1,58 @@
# Copyright 2015 Hewlett-Packard Development Company, L.P.
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
import logging
import voluptuous as v
from zuul.model import EventFilter
from zuul.trigger import BaseTrigger
class GithubTrigger(BaseTrigger):
name = 'github'
log = logging.getLogger("zuul.trigger.GithubTrigger")
def _toList(self, item):
if not item:
return []
if isinstance(item, list):
return item
return [item]
def getEventFilters(self, trigger_config):
efilters = []
for trigger in self._toList(trigger_config):
types = trigger.get('event', None)
actions = trigger.get('action')
f = EventFilter(trigger=self,
types=self._toList(types),
actions=self._toList(actions))
efilters.append(f)
return efilters
def onPullRequest(self, payload):
pass
def getSchema():
def toList(x):
return v.Any([x], x)
github_trigger = {
v.Required('event'):
toList(v.Any('pull_request')),
'action': toList(str),
}
return github_trigger

View File

@ -18,6 +18,7 @@ import re
import zuul.driver.zuul import zuul.driver.zuul
import zuul.driver.gerrit import zuul.driver.gerrit
import zuul.driver.git import zuul.driver.git
import zuul.driver.github
import zuul.driver.smtp import zuul.driver.smtp
import zuul.driver.timer import zuul.driver.timer
import zuul.driver.sql import zuul.driver.sql
@ -40,6 +41,7 @@ class ConnectionRegistry(object):
self.registerDriver(zuul.driver.zuul.ZuulDriver()) self.registerDriver(zuul.driver.zuul.ZuulDriver())
self.registerDriver(zuul.driver.gerrit.GerritDriver()) self.registerDriver(zuul.driver.gerrit.GerritDriver())
self.registerDriver(zuul.driver.git.GitDriver()) self.registerDriver(zuul.driver.git.GitDriver())
self.registerDriver(zuul.driver.github.GithubDriver())
self.registerDriver(zuul.driver.smtp.SMTPDriver()) self.registerDriver(zuul.driver.smtp.SMTPDriver())
self.registerDriver(zuul.driver.timer.TimerDriver()) self.registerDriver(zuul.driver.timer.TimerDriver())
self.registerDriver(zuul.driver.sql.SQLDriver()) self.registerDriver(zuul.driver.sql.SQLDriver())

View File

@ -1822,7 +1822,7 @@ class Change(Ref):
def getBasePath(self): def getBasePath(self):
if hasattr(self, 'refspec'): if hasattr(self, 'refspec'):
return "%s/%s/%s" % ( return "%s/%s/%s" % (
self.number[-2:], self.number, self.patchset) str(self.number)[-2:], self.number, self.patchset)
return super(Change, self).getBasePath() return super(Change, self).getBasePath()
def equals(self, other): def equals(self, other):
@ -1859,6 +1859,20 @@ class Change(Ref):
patchset=self.patchset) patchset=self.patchset)
class PullRequest(Change):
def __init__(self, project):
super(PullRequest, self).__init__(project)
self.updated_at = None
def isUpdateOf(self, other):
if (hasattr(other, 'number') and self.number == other.number and
hasattr(other, 'patchset') and self.patchset != other.patchset and
hasattr(other, 'updated_at') and
self.updated_at > other.updated_at):
return True
return False
class TriggerEvent(object): class TriggerEvent(object):
"""Incoming event from an external system.""" """Incoming event from an external system."""
def __init__(self): def __init__(self):
@ -2008,7 +2022,7 @@ class EventFilter(BaseFilter):
def __init__(self, trigger, types=[], branches=[], refs=[], def __init__(self, trigger, types=[], branches=[], refs=[],
event_approvals={}, comments=[], emails=[], usernames=[], event_approvals={}, comments=[], emails=[], usernames=[],
timespecs=[], required_approvals=[], reject_approvals=[], timespecs=[], required_approvals=[], reject_approvals=[],
pipelines=[], ignore_deletes=True): pipelines=[], actions=[], ignore_deletes=True):
super(EventFilter, self).__init__( super(EventFilter, self).__init__(
required_approvals=required_approvals, required_approvals=required_approvals,
reject_approvals=reject_approvals) reject_approvals=reject_approvals)
@ -2027,6 +2041,7 @@ class EventFilter(BaseFilter):
self.emails = [re.compile(x) for x in emails] self.emails = [re.compile(x) for x in emails]
self.usernames = [re.compile(x) for x in usernames] self.usernames = [re.compile(x) for x in usernames]
self.pipelines = [re.compile(x) for x in pipelines] self.pipelines = [re.compile(x) for x in pipelines]
self.actions = actions
self.event_approvals = event_approvals self.event_approvals = event_approvals
self.timespecs = timespecs self.timespecs = timespecs
self.ignore_deletes = ignore_deletes self.ignore_deletes = ignore_deletes
@ -2061,6 +2076,8 @@ class EventFilter(BaseFilter):
ret += ' username_filters: %s' % ', '.join(self._usernames) ret += ' username_filters: %s' % ', '.join(self._usernames)
if self.timespecs: if self.timespecs:
ret += ' timespecs: %s' % ', '.join(self.timespecs) ret += ' timespecs: %s' % ', '.join(self.timespecs)
if self.actions:
ret += ' actions: %s' % ', '.join(self.actions)
ret += '>' ret += '>'
return ret return ret
@ -2157,6 +2174,14 @@ class EventFilter(BaseFilter):
if self.timespecs and not matches_timespec: if self.timespecs and not matches_timespec:
return False return False
# actions are ORed
matches_action = False
for action in self.actions:
if (event.action == action):
matches_action = True
if self.actions and not matches_action:
return False
return True return True

View File

@ -45,7 +45,7 @@ array of changes, they will not include the queue structure.
class WebApp(threading.Thread): class WebApp(threading.Thread):
log = logging.getLogger("zuul.WebApp") log = logging.getLogger("zuul.WebApp")
change_path_regexp = '/status/change/(\d+,\d+)$' change_path_regexp = '/status/change/(.*)$'
def __init__(self, scheduler, port=8001, cache_expiry=1, def __init__(self, scheduler, port=8001, cache_expiry=1,
listen_address='0.0.0.0'): listen_address='0.0.0.0'):