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>
diff --git a/doc/source/connections.rst b/doc/source/connections.rst
index 614b44a..521e4ed 100644
--- a/doc/source/connections.rst
+++ b/doc/source/connections.rst
@@ -65,6 +65,24 @@
 be added to Gerrit.  Zuul is very flexible and can take advantage of
 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
 ----
 
diff --git a/doc/source/triggers.rst b/doc/source/triggers.rst
index 263f280..0053f64 100644
--- a/doc/source/triggers.rst
+++ b/doc/source/triggers.rst
@@ -4,7 +4,7 @@
 ========
 
 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
 systems can be supported.
 
@@ -100,6 +100,38 @@
   *require-approval* but will fail to enter the pipeline if there is
   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
 -----
@@ -154,4 +186,4 @@
   *reject-approval*
   This takes a list of approvals in the same format as
   *require-approval* but will fail to enter the pipeline if there is
-  a matching approval.
\ No newline at end of file
+  a matching approval.
diff --git a/requirements.txt b/requirements.txt
index 974b77f..44cef95 100644
--- a/requirements.txt
+++ b/requirements.txt
@@ -1,5 +1,6 @@
 pbr>=1.1.0
 
+Github3.py
 PyYAML>=3.1.0
 Paste
 WebOb>=1.2.3
diff --git a/tests/base.py b/tests/base.py
index 2c3f7bb..e33b510 100755
--- a/tests/base.py
+++ b/tests/base.py
@@ -58,6 +58,7 @@
 
 import zuul.driver.gerrit.gerritsource as gerritsource
 import zuul.driver.gerrit.gerritconnection as gerritconnection
+import zuul.driver.github.githubconnection as githubconnection
 import zuul.scheduler
 import zuul.webapp
 import zuul.rpclistener
@@ -126,12 +127,12 @@
     return decorator
 
 
-class ChangeReference(git.Reference):
+class GerritChangeReference(git.Reference):
     _common_path_default = "refs/changes"
     _points_to_commits_only = True
 
 
-class FakeChange(object):
+class FakeGerritChange(object):
     categories = {'approved': ('Approved', -1, 1),
                   'code-review': ('Code-Review', -2, 2),
                   'verified': ('Verified', -2, 2)}
@@ -139,6 +140,7 @@
     def __init__(self, gerrit, number, project, branch, subject,
                  status='NEW', upstream_root=None, files={}):
         self.gerrit = gerrit
+        self.source = gerrit
         self.reported = 0
         self.queried = 0
         self.patchsets = []
@@ -178,9 +180,9 @@
     def addFakeChangeToRepo(self, msg, files, large):
         path = os.path.join(self.upstream_root, self.project)
         repo = git.Repo(path)
-        ref = ChangeReference.create(repo, '1/%s/%s' % (self.number,
-                                                        self.latest_patchset),
-                                     'refs/tags/init')
+        ref = GerritChangeReference.create(
+            repo, '1/%s/%s' % (self.number, self.latest_patchset),
+            'refs/tags/init')
         repo.head.reference = ref
         zuul.merger.merger.reset_repo_to_head(repo)
         repo.git.clean('-x', '-f', '-d')
@@ -469,9 +471,9 @@
                       files=None):
         """Add a change to the fake Gerrit."""
         self.change_number += 1
-        c = FakeChange(self, self.change_number, project, branch, subject,
-                       upstream_root=self.upstream_root,
-                       status=status, files=files)
+        c = FakeGerritChange(self, self.change_number, project, branch,
+                             subject, upstream_root=self.upstream_root,
+                             status=status, files=files)
         self.changes[self.change_number] = c
         return c
 
@@ -536,6 +538,162 @@
         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):
     def __init__(self, **kw):
         self.__dict__.update(kw)
@@ -701,7 +859,7 @@
 
         """
         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)
             try:
                 repo = git.Repo(path)
@@ -1451,6 +1609,16 @@
             'zuul.driver.gerrit.GerritDriver.getConnection',
             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
         # TODO(jhesketh): This should come from lib.connections for better
         # coverage
diff --git a/tests/fixtures/layouts/basic-github.yaml b/tests/fixtures/layouts/basic-github.yaml
new file mode 100644
index 0000000..79d416a
--- /dev/null
+++ b/tests/fixtures/layouts/basic-github.yaml
@@ -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
diff --git a/tests/fixtures/zuul-github-driver.conf b/tests/fixtures/zuul-github-driver.conf
new file mode 100644
index 0000000..b979a3f
--- /dev/null
+++ b/tests/fixtures/zuul-github-driver.conf
@@ -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
diff --git a/tests/unit/test_github_driver.py b/tests/unit/test_github_driver.py
new file mode 100644
index 0000000..58f456f
--- /dev/null
+++ b/tests/unit/test_github_driver.py
@@ -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'])
diff --git a/zuul/driver/github/__init__.py b/zuul/driver/github/__init__.py
new file mode 100644
index 0000000..af29b62
--- /dev/null
+++ b/zuul/driver/github/__init__.py
@@ -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()
diff --git a/zuul/driver/github/githubconnection.py b/zuul/driver/github/githubconnection.py
new file mode 100644
index 0000000..3878949
--- /dev/null
+++ b/zuul/driver/github/githubconnection.py
@@ -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
diff --git a/zuul/driver/github/githubsource.py b/zuul/driver/github/githubsource.py
new file mode 100644
index 0000000..1464f8a
--- /dev/null
+++ b/zuul/driver/github/githubsource.py
@@ -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')
diff --git a/zuul/driver/github/githubtrigger.py b/zuul/driver/github/githubtrigger.py
new file mode 100644
index 0000000..17150bc
--- /dev/null
+++ b/zuul/driver/github/githubtrigger.py
@@ -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
diff --git a/zuul/lib/connections.py b/zuul/lib/connections.py
index 403aca6..f5cce7b 100644
--- a/zuul/lib/connections.py
+++ b/zuul/lib/connections.py
@@ -18,6 +18,7 @@
 import zuul.driver.zuul
 import zuul.driver.gerrit
 import zuul.driver.git
+import zuul.driver.github
 import zuul.driver.smtp
 import zuul.driver.timer
 import zuul.driver.sql
@@ -40,6 +41,7 @@
         self.registerDriver(zuul.driver.zuul.ZuulDriver())
         self.registerDriver(zuul.driver.gerrit.GerritDriver())
         self.registerDriver(zuul.driver.git.GitDriver())
+        self.registerDriver(zuul.driver.github.GithubDriver())
         self.registerDriver(zuul.driver.smtp.SMTPDriver())
         self.registerDriver(zuul.driver.timer.TimerDriver())
         self.registerDriver(zuul.driver.sql.SQLDriver())
diff --git a/zuul/model.py b/zuul/model.py
index af80028..3f56a3a 100644
--- a/zuul/model.py
+++ b/zuul/model.py
@@ -1822,7 +1822,7 @@
     def getBasePath(self):
         if hasattr(self, 'refspec'):
             return "%s/%s/%s" % (
-                self.number[-2:], self.number, self.patchset)
+                str(self.number)[-2:], self.number, self.patchset)
         return super(Change, self).getBasePath()
 
     def equals(self, other):
@@ -1859,6 +1859,20 @@
                           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):
     """Incoming event from an external system."""
     def __init__(self):
@@ -2008,7 +2022,7 @@
     def __init__(self, trigger, types=[], branches=[], refs=[],
                  event_approvals={}, comments=[], emails=[], usernames=[],
                  timespecs=[], required_approvals=[], reject_approvals=[],
-                 pipelines=[], ignore_deletes=True):
+                 pipelines=[], actions=[], ignore_deletes=True):
         super(EventFilter, self).__init__(
             required_approvals=required_approvals,
             reject_approvals=reject_approvals)
@@ -2027,6 +2041,7 @@
         self.emails = [re.compile(x) for x in emails]
         self.usernames = [re.compile(x) for x in usernames]
         self.pipelines = [re.compile(x) for x in pipelines]
+        self.actions = actions
         self.event_approvals = event_approvals
         self.timespecs = timespecs
         self.ignore_deletes = ignore_deletes
@@ -2061,6 +2076,8 @@
             ret += ' username_filters: %s' % ', '.join(self._usernames)
         if self.timespecs:
             ret += ' timespecs: %s' % ', '.join(self.timespecs)
+        if self.actions:
+            ret += ' actions: %s' % ', '.join(self.actions)
         ret += '>'
 
         return ret
@@ -2157,6 +2174,14 @@
         if self.timespecs and not matches_timespec:
             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
 
 
diff --git a/zuul/webapp.py b/zuul/webapp.py
index f5a7373..37d6ddd 100644
--- a/zuul/webapp.py
+++ b/zuul/webapp.py
@@ -45,7 +45,7 @@
 
 class WebApp(threading.Thread):
     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,
                  listen_address='0.0.0.0'):