Merge "Add an internals doc to the documentation" into feature/zuulv3
diff --git a/doc/source/launchers.rst b/doc/source/launchers.rst
index f368cb9..78d5839 100644
--- a/doc/source/launchers.rst
+++ b/doc/source/launchers.rst
@@ -362,24 +362,3 @@
 
 The original job is expected to complete with a WORK_DATA and
 WORK_FAIL packet as described in `Starting Builds`_.
-
-Build Descriptions
-^^^^^^^^^^^^^^^^^^
-
-In order to update the job running system with a description of the
-current state of all related builds, the job runner may optionally
-implement the following Gearman function:
-
-  set_description:MANAGER_NAME
-
-Where **MANAGER_NAME** is used as described in `Stopping Builds`_.
-The argument to the function is the following encoded in JSON format:
-
-**name**
-  The job name of the build to describe.
-
-**number**
-  The build number of the build to describe.
-
-**html_description**
-  The description of the build in HTML format.
diff --git a/doc/source/zuul.rst b/doc/source/zuul.rst
index 2285ecb..102beac 100644
--- a/doc/source/zuul.rst
+++ b/doc/source/zuul.rst
@@ -52,11 +52,6 @@
   Port on which the Gearman server is listening.
   ``port=4730`` (optional)
 
-**check_job_registration**
-  Check to see if job is registered with Gearman or not. When True
-  a build result of NOT_REGISTERED will be return if job is not found.
-  ``check_job_registration=True``
-
 gearman_server
 """"""""""""""
 
diff --git a/tests/base.py b/tests/base.py
index a75d36b..e1e568f 100755
--- a/tests/base.py
+++ b/tests/base.py
@@ -529,25 +529,26 @@
         os.write(self.wake_write, '1\n')
 
 
-class FakeBuild(threading.Thread):
+class FakeBuild(object):
     log = logging.getLogger("zuul.test")
 
-    def __init__(self, worker, job, number, node):
-        threading.Thread.__init__(self)
+    def __init__(self, launch_server, job, number, node):
         self.daemon = True
-        self.worker = worker
+        self.launch_server = launch_server
         self.job = job
-        self.name = job.name.split(':')[1]
         self.number = number
         self.node = node
         self.parameters = json.loads(job.arguments)
         self.unique = self.parameters['ZUUL_UUID']
+        self.name = self.parameters['job']
         self.wait_condition = threading.Condition()
         self.waiting = False
         self.aborted = False
         self.created = time.time()
-        self.description = ''
         self.run_error = False
+        self.changes = None
+        if 'ZUUL_CHANGE_IDS' in self.parameters:
+            self.changes = self.parameters['ZUUL_CHANGE_IDS']
 
     def release(self):
         self.wait_condition.acquire()
@@ -577,7 +578,7 @@
             'url': 'https://server/job/%s/%s/' % (self.name, self.number),
             'name': self.name,
             'number': self.number,
-            'manager': self.worker.worker_id,
+            'manager': self.launch_server.worker.worker_id,
             'worker_name': 'My Worker',
             'worker_hostname': 'localhost',
             'worker_ips': ['127.0.0.1', '192.168.1.1'],
@@ -593,79 +594,87 @@
         self.log.debug('Sent WorkData packet with %s' % json.dumps(data))
         self.job.sendWorkStatus(0, 100)
 
-        if self.worker.hold_jobs_in_build:
+        if self.launch_server.hold_jobs_in_build:
             self.log.debug('Holding build %s' % self.unique)
             self._wait()
         self.log.debug("Build %s continuing" % self.unique)
 
-        self.worker.lock.acquire()
-
         result = 'SUCCESS'
         if (('ZUUL_REF' in self.parameters) and
-            self.worker.shouldFailTest(self.name,
-                                       self.parameters['ZUUL_REF'])):
+            self.launch_server.shouldFailTest(self.name,
+                                              self.parameters['ZUUL_REF'])):
             result = 'FAILURE'
         if self.aborted:
             result = 'ABORTED'
 
         if self.run_error:
-            work_fail = True
             result = 'RUN_ERROR'
-        else:
-            data['result'] = result
-            data['node_labels'] = ['bare-necessities']
-            data['node_name'] = 'foo'
-            work_fail = False
 
-        changes = None
-        if 'ZUUL_CHANGE_IDS' in self.parameters:
-            changes = self.parameters['ZUUL_CHANGE_IDS']
-
-        self.worker.build_history.append(
-            BuildHistory(name=self.name, number=self.number,
-                         result=result, changes=changes, node=self.node,
-                         uuid=self.unique, description=self.description,
-                         parameters=self.parameters,
-                         pipeline=self.parameters['ZUUL_PIPELINE'])
-        )
-
-        self.job.sendWorkData(json.dumps(data))
-        if work_fail:
-            self.job.sendWorkFail()
-        else:
-            self.job.sendWorkComplete(json.dumps(data))
-        del self.worker.gearman_jobs[self.job.unique]
-        self.worker.running_builds.remove(self)
-        self.worker.lock.release()
+        return result
 
 
 class RecordingLaunchServer(zuul.launcher.server.LaunchServer):
     def __init__(self, *args, **kw):
+        self._run_ansible = kw.pop('_run_ansible', False)
         super(RecordingLaunchServer, self).__init__(*args, **kw)
-        self.job_history = []
+        self.hold_jobs_in_build = False
+        self.lock = threading.Lock()
+        self.running_builds = []
+        self.build_history = []
+        self._build_counter_lock = threading.Lock()
+        self.build_counter = 0
+        self.fail_tests = {}
 
-    def launch(self, job):
-        self.job_history.append(job)
-        job.data = []
+    def addFailTest(self, name, change):
+        l = self.fail_tests.get(name, [])
+        l.append(change)
+        self.fail_tests[name] = l
 
-        def sendWorkComplete(data=b''):
-            job.data.append(data)
-            gear.WorkerJob.sendWorkComplete(job, data)
+    def shouldFailTest(self, name, ref):
+        l = self.fail_tests.get(name, [])
+        for change in l:
+            if self.test.ref_has_change(ref, change):
+                return True
+        return False
 
-        job.sendWorkComplete = sendWorkComplete
-        super(RecordingLaunchServer, self).launch(job)
+    def runAnsible(self, jobdir, job):
+        with self._build_counter_lock:
+            self.build_counter += 1
+            build_counter = self.build_counter
+        node = None
+        build = FakeBuild(self, job, build_counter, node)
+        job.build = build
+
+        self.running_builds.append(build)
+
+        if self._run_ansible:
+            result = super(RecordingLaunchServer, self).runAnsible(jobdir, job)
+        else:
+            result = build.run()
+
+        self.lock.acquire()
+        self.build_history.append(
+            BuildHistory(name=build.name, number=build.number,
+                         result=result, changes=build.changes, node=build.node,
+                         uuid=build.unique, parameters=build.parameters,
+                         pipeline=build.parameters['ZUUL_PIPELINE'])
+        )
+        if build:
+            self.running_builds.remove(build)
+        self.lock.release()
+        return result
 
 
 class FakeWorker(gear.Worker):
     def __init__(self, worker_id, test):
         super(FakeWorker, self).__init__(worker_id)
-        self.gearman_jobs = {}
         self.build_history = []
         self.running_builds = []
         self.build_counter = 0
         self.fail_tests = {}
         self.test = test
 
+        self.registerFunction('launcher:launch')
         self.hold_jobs_in_build = False
         self.lock = threading.Lock()
         self.__work_thread = threading.Thread(target=self.work)
@@ -674,29 +683,23 @@
 
     def handleJob(self, job):
         parts = job.name.split(":")
-        cmd = parts[0]
-        name = parts[1]
-        if len(parts) > 2:
-            node = parts[2]
-        else:
-            node = None
-        if cmd == 'build':
-            self.handleBuild(job, name, node)
+        cmd = parts[1]
+        if cmd == 'launch':
+            self.handleLaunch(job)
         elif cmd == 'stop':
-            self.handleStop(job, name)
-        elif cmd == 'set_description':
-            self.handleSetDescription(job, name)
+            self.handleStop(job)
 
-    def handleBuild(self, job, name, node):
+    def handleLaunch(self, job):
+        # TODOv3(jeblair): handle nodes
+        node = None
         build = FakeBuild(self, job, self.build_counter, node)
         job.build = build
-        self.gearman_jobs[job.unique] = job
         self.build_counter += 1
 
         self.running_builds.append(build)
         build.start()
 
-    def handleStop(self, job, name):
+    def handleStop(self, job):
         self.log.debug("handle stop")
         parameters = json.loads(job.arguments)
         name = parameters['name']
@@ -709,24 +712,6 @@
                 return
         job.sendWorkFail()
 
-    def handleSetDescription(self, job, name):
-        self.log.debug("handle set description")
-        parameters = json.loads(job.arguments)
-        name = parameters['name']
-        number = parameters['number']
-        descr = parameters['html_description']
-        for build in self.running_builds:
-            if build.name == name and build.number == number:
-                build.description = descr
-                job.sendWorkComplete()
-                return
-        for build in self.build_history:
-            if build.name == name and build.number == number:
-                build.description = descr
-                job.sendWorkComplete()
-                return
-        job.sendWorkFail()
-
     def work(self):
         while self.running:
             try:
@@ -909,6 +894,12 @@
 
 class ZuulTestCase(BaseTestCase):
     config_file = 'zuul.conf'
+    run_ansible = False
+
+    def _startMerger(self):
+        self.merge_server = zuul.merger.server.MergeServer(self.config,
+                                                           self.connections)
+        self.merge_server.start()
 
     def setUp(self):
         super(ZuulTestCase, self).setUp()
@@ -988,10 +979,6 @@
         self.configure_connections()
         self.sched.registerConnections(self.connections)
 
-        self.ansible_server = RecordingLaunchServer(
-            self.config, self.connections)
-        self.ansible_server.start()
-
         def URLOpenerFactory(*args, **kw):
             if isinstance(args[0], urllib.request.Request):
                 return old_urlopen(*args, **kw)
@@ -1000,13 +987,21 @@
         old_urlopen = urllib.request.urlopen
         urllib.request.urlopen = URLOpenerFactory
 
-        self.launcher = zuul.launcher.client.LaunchClient(
+        self._startMerger()
+
+        self.launch_server = RecordingLaunchServer(
+            self.config, self.connections, _run_ansible=self.run_ansible)
+        self.launch_server.start()
+        self.history = self.launch_server.build_history
+        self.builds = self.launch_server.running_builds
+
+        self.launch_client = zuul.launcher.client.LaunchClient(
             self.config, self.sched, self.swift)
         self.merge_client = zuul.merger.client.MergeClient(
             self.config, self.sched)
         self.nodepool = zuul.nodepool.Nodepool(self.sched)
 
-        self.sched.setLauncher(self.launcher)
+        self.sched.setLauncher(self.launch_client)
         self.sched.setMerger(self.merge_client)
         self.sched.setNodepool(self.nodepool)
 
@@ -1019,7 +1014,7 @@
         self.sched.resume()
         self.webapp.start()
         self.rpc.start()
-        self.launcher.gearman.waitForServer()
+        self.launch_client.gearman.waitForServer()
 
         self.addCleanup(self.assertFinalState)
         self.addCleanup(self.shutdown)
@@ -1148,8 +1143,11 @@
 
     def shutdown(self):
         self.log.debug("Shutting down after tests")
-        self.launcher.stop()
+        self.launch_client.stop()
+        self.merge_server.stop()
+        self.merge_server.join()
         self.merge_client.stop()
+        self.launch_server.stop()
         self.sched.stop()
         self.sched.join()
         self.statsd.stop()
@@ -1271,7 +1269,7 @@
             return parameters[name]
 
     def resetGearmanServer(self):
-        self.worker.setFunctions([])
+        self.launch_server.worker.setFunctions([])
         while True:
             done = True
             for connection in self.gearman_server.active_connections:
@@ -1287,29 +1285,29 @@
 
     def haveAllBuildsReported(self):
         # See if Zuul is waiting on a meta job to complete
-        if self.launcher.meta_jobs:
+        if self.launch_client.meta_jobs:
             return False
         # Find out if every build that the worker has completed has been
         # reported back to Zuul.  If it hasn't then that means a Gearman
         # event is still in transit and the system is not stable.
-        for job in self.ansible_server.job_history:
-            zbuild = self.launcher.builds.get(job.unique)
+        for build in self.history:
+            zbuild = self.launch_client.builds.get(build.uuid)
             if not zbuild:
                 # It has already been reported
                 continue
             # It hasn't been reported yet.
             return False
         # Make sure that none of the worker connections are in GRAB_WAIT
-        for connection in self.ansible_server.worker.active_connections:
+        for connection in self.launch_server.worker.active_connections:
             if connection.state == 'GRAB_WAIT':
                 return False
         return True
 
     def areAllBuildsWaiting(self):
-        builds = self.launcher.builds.values()
+        builds = self.launch_client.builds.values()
         for build in builds:
             client_job = None
-            for conn in self.launcher.gearman.active_connections:
+            for conn in self.launch_client.gearman.active_connections:
                 for j in conn.related_jobs.values():
                     if j.unique == build.uuid:
                         client_job = j
@@ -1363,6 +1361,8 @@
                                (self.areAllBuildsWaiting(),))
                 raise Exception("Timeout waiting for Zuul to settle")
             # Make sure no new events show up while we're checking
+
+            self.launch_server.lock.acquire()
             # have all build states propogated to zuul?
             if self.haveAllBuildsReported():
                 # Join ensures that the queue is empty _and_ events have been
@@ -1374,9 +1374,11 @@
                     self.haveAllBuildsReported() and
                     self.areAllBuildsWaiting()):
                     self.sched.run_handler_lock.release()
+                    self.launch_server.lock.release()
                     self.log.debug("...settled.")
                     return
                 self.sched.run_handler_lock.release()
+            self.launch_server.lock.release()
             self.sched.wake_event.wait(0.1)
 
     def countJobResults(self, jobs, result):
@@ -1384,16 +1386,11 @@
         return len(jobs)
 
     def getJobFromHistory(self, name, project=None):
-        history = self.ansible_server.job_history
-        for job in history:
-            params = json.loads(job.arguments)
-            if (params['job'] == name and
-                (project is None or params['ZUUL_PROJECT'] == project)):
-                result = json.loads(job.data[-1])
-                ret = BuildHistory(job=job,
-                                   name=params['job'],
-                                   result=result['result'])
-                return ret
+        for job in self.history:
+            if (job.name == name and
+                (project is None or
+                 job.parameters['ZUUL_PROJECT'] == project)):
+                return job
         raise Exception("Unable to find job %s in history" % name)
 
     def assertEmptyQueues(self):
@@ -1461,3 +1458,8 @@
         repo.heads[branch].checkout()
         if tag:
             repo.create_tag(tag)
+
+
+class AnsibleZuulTestCase(ZuulTestCase):
+    """ZuulTestCase but with an actual ansible launcher running"""
+    run_ansible = True
diff --git a/tests/cmd/__init__.py b/tests/cmd/__init__.py
deleted file mode 100644
index e69de29..0000000
--- a/tests/cmd/__init__.py
+++ /dev/null
diff --git a/tests/fixtures/config/single-tenant/git/common-config/zuul.yaml b/tests/fixtures/config/single-tenant/git/common-config/zuul.yaml
new file mode 100644
index 0000000..d316aab
--- /dev/null
+++ b/tests/fixtures/config/single-tenant/git/common-config/zuul.yaml
@@ -0,0 +1,58 @@
+- pipeline:
+    name: check
+    manager: independent
+    source:
+      gerrit
+    trigger:
+      gerrit:
+        - event: patchset-created
+    success:
+      gerrit:
+        verified: 1
+    failure:
+      gerrit:
+        verified: -1
+
+- pipeline:
+    name: gate
+    manager: dependent
+    success-message: Build succeeded (gate).
+    source:
+      gerrit
+    trigger:
+      gerrit:
+        - event: comment-added
+          approval:
+            - approved: 1
+    success:
+      gerrit:
+        verified: 2
+        submit: true
+    failure:
+      gerrit:
+        verified: -2
+    start:
+      gerrit:
+        verified: 0
+    precedence: high
+
+- job:
+    name:
+      project-merge
+
+- job:
+    name:
+      project-test1
+
+- job:
+    name:
+      project-test2
+
+- project:
+    name: org/project
+    gate:
+      jobs:
+        - project-merge:
+            jobs:
+              - project-test1
+              - project-test2
diff --git a/tests/fixtures/config/single-tenant/git/org_project/README b/tests/fixtures/config/single-tenant/git/org_project/README
new file mode 100644
index 0000000..9daeafb
--- /dev/null
+++ b/tests/fixtures/config/single-tenant/git/org_project/README
@@ -0,0 +1 @@
+test
diff --git a/tests/fixtures/config/single-tenant/main.yaml b/tests/fixtures/config/single-tenant/main.yaml
new file mode 100644
index 0000000..a22ed5c
--- /dev/null
+++ b/tests/fixtures/config/single-tenant/main.yaml
@@ -0,0 +1,6 @@
+- tenant:
+    name: tenant-one
+    source:
+      gerrit:
+        config-repos:
+          - common-config
diff --git a/tests/print_layout.py b/tests/print_layout.py
new file mode 100644
index 0000000..9afd379
--- /dev/null
+++ b/tests/print_layout.py
@@ -0,0 +1,65 @@
+#!/usr/bin/env python
+
+# 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 argparse
+import os
+import sys
+
+import tests.base
+
+CONFIG_DIR = os.path.join(tests.base.FIXTURE_DIR, 'config')
+
+
+def print_file(title, path):
+    print('')
+    print(title)
+    print('-' * 78)
+    with open(path) as f:
+        print(f.read())
+    print('-' * 78)
+
+
+def main():
+    parser = argparse.ArgumentParser(description='Print test layout.')
+    parser.add_argument(dest='config', nargs='?',
+                        help='the test configuration name')
+    args = parser.parse_args()
+    if not args.config:
+        print('Available test configurations:')
+        for d in os.listdir(CONFIG_DIR):
+            print('  ' + d)
+        sys.exit(1)
+    configdir = os.path.join(CONFIG_DIR, args.config)
+
+    title = '   Configuration: %s   ' % args.config
+    print('=' * len(title))
+    print(title)
+    print('=' * len(title))
+    print_file('Main Configuration',
+               os.path.join(configdir, 'main.yaml'))
+
+    gitroot = os.path.join(configdir, 'git')
+    for gitrepo in os.listdir(gitroot):
+        reporoot = os.path.join(gitroot, gitrepo)
+        print('')
+        print('=== Git repo: %s ===' % gitrepo)
+        filenames = os.listdir(reporoot)
+        for fn in filenames:
+            if fn in ['zuul.yaml', '.zuul.yaml']:
+                print_file('File: ' + os.path.join(gitrepo, fn),
+                           os.path.join(reporoot, fn))
+
+
+if __name__ == '__main__':
+    main()
diff --git a/tests/cmd/test_cloner.py b/tests/test_cloner_cmd.py
similarity index 100%
rename from tests/cmd/test_cloner.py
rename to tests/test_cloner_cmd.py
diff --git a/tests/test_model.py b/tests/test_model.py
index 99cc57f..fa670a4 100644
--- a/tests/test_model.py
+++ b/tests/test_model.py
@@ -120,6 +120,81 @@
         self.assertEqual(job.name, 'python27')
         self.assertEqual(job.timeout, 50)
 
+    def test_job_auth_inheritance(self):
+        layout = model.Layout()
+        project = model.Project('project')
+
+        base = configloader.JobParser.fromYaml(layout, {
+            '_source_project': project,
+            'name': 'base',
+            'timeout': 30,
+        })
+        layout.addJob(base)
+        pypi_upload_without_inherit = configloader.JobParser.fromYaml(layout, {
+            '_source_project': project,
+            'name': 'pypi-upload-without-inherit',
+            'parent': 'base',
+            'timeout': 40,
+            'auth': {
+                'password': {
+                    'pypipassword': 'dummypassword'
+                }
+            }
+        })
+        layout.addJob(pypi_upload_without_inherit)
+        pypi_upload_with_inherit = configloader.JobParser.fromYaml(layout, {
+            '_source_project': project,
+            'name': 'pypi-upload-with-inherit',
+            'parent': 'base',
+            'timeout': 40,
+            'auth': {
+                'inherit': True,
+                'password': {
+                    'pypipassword': 'dummypassword'
+                }
+            }
+        })
+        layout.addJob(pypi_upload_with_inherit)
+        pypi_upload_with_inherit_false = configloader.JobParser.fromYaml(
+            layout, {
+                '_source_project': project,
+                'name': 'pypi-upload-with-inherit-false',
+                'parent': 'base',
+                'timeout': 40,
+                'auth': {
+                    'inherit': False,
+                    'password': {
+                        'pypipassword': 'dummypassword'
+                    }
+                }
+            })
+        layout.addJob(pypi_upload_with_inherit_false)
+        in_repo_job_without_inherit = configloader.JobParser.fromYaml(layout, {
+            '_source_project': project,
+            'name': 'in-repo-job-without-inherit',
+            'parent': 'pypi-upload-without-inherit',
+        })
+        layout.addJob(in_repo_job_without_inherit)
+        in_repo_job_with_inherit = configloader.JobParser.fromYaml(layout, {
+            '_source_project': project,
+            'name': 'in-repo-job-with-inherit',
+            'parent': 'pypi-upload-with-inherit',
+        })
+        layout.addJob(in_repo_job_with_inherit)
+        in_repo_job_with_inherit_false = configloader.JobParser.fromYaml(
+            layout, {
+                '_source_project': project,
+                'name': 'in-repo-job-with-inherit-false',
+                'parent': 'pypi-upload-with-inherit-false',
+            })
+        layout.addJob(in_repo_job_with_inherit_false)
+
+        self.assertNotIn('auth', in_repo_job_without_inherit.auth)
+        self.assertIn('password', in_repo_job_with_inherit.auth)
+        self.assertEquals(in_repo_job_with_inherit.auth['password'],
+                          {'pypipassword': 'dummypassword'})
+        self.assertNotIn('auth', in_repo_job_with_inherit_false.auth)
+
     def test_job_inheritance_job_tree(self):
         layout = model.Layout()
 
diff --git a/tests/test_scheduler.py b/tests/test_scheduler.py
index df6bc1b..716ec9e 100755
--- a/tests/test_scheduler.py
+++ b/tests/test_scheduler.py
@@ -21,6 +21,7 @@
 import shutil
 import time
 import yaml
+from unittest import skip
 
 import git
 from six.moves import urllib
@@ -33,7 +34,6 @@
 import zuul.reporter.smtp
 
 from tests.base import (
-    BaseTestCase,
     ZuulTestCase,
     repack_repo,
 )
@@ -43,47 +43,8 @@
                     '%(levelname)-8s %(message)s')
 
 
-class TestSchedulerConfigParsing(BaseTestCase):
-
-    def setUp(self):
-        self.skip("Disabled for early v3 development")
-
-    def test_parse_skip_if(self):
-        job_yaml = """
-jobs:
-  - name: job_name
-    skip-if:
-      - project: ^project_name$
-        branch: ^stable/icehouse$
-        all-files-match-any:
-          - ^filename$
-      - project: ^project2_name$
-        all-files-match-any:
-          - ^filename2$
-    """.strip()
-        data = yaml.load(job_yaml)
-        config_job = data.get('jobs')[0]
-        sched = zuul.scheduler.Scheduler({})
-        cm = zuul.change_matcher
-        expected = cm.MatchAny([
-            cm.MatchAll([
-                cm.ProjectMatcher('^project_name$'),
-                cm.BranchMatcher('^stable/icehouse$'),
-                cm.MatchAllFiles([cm.FileMatcher('^filename$')]),
-            ]),
-            cm.MatchAll([
-                cm.ProjectMatcher('^project2_name$'),
-                cm.MatchAllFiles([cm.FileMatcher('^filename2$')]),
-            ]),
-        ])
-        matcher = sched._parseSkipIf(config_job)
-        self.assertEqual(expected, matcher)
-
-
 class TestScheduler(ZuulTestCase):
-
-    def setUp(self):
-        self.skip("Disabled for early v3 development")
+    tenant_config_file = 'config/single-tenant/main.yaml'
 
     def test_jobs_launched(self):
         "Test that jobs are launched and a change is merged"
@@ -101,6 +62,7 @@
         self.assertEqual(A.data['status'], 'MERGED')
         self.assertEqual(A.reported, 2)
 
+        # TODOv3(jeblair): we may want to report stats by tenant (also?).
         self.assertReportedStat('gerrit.event.comment-added', value='1|c')
         self.assertReportedStat('zuul.pipeline.gate.current_changes',
                                 value='1|g')
@@ -121,12 +83,12 @@
 
     def test_initial_pipeline_gauges(self):
         "Test that each pipeline reported its length on start"
-        pipeline_names = self.sched.layout.pipelines.keys()
-        self.assertNotEqual(len(pipeline_names), 0)
-        for name in pipeline_names:
-            self.assertReportedStat('zuul.pipeline.%s.current_changes' % name,
-                                    value='0|g')
+        self.assertReportedStat('zuul.pipeline.gate.current_changes',
+                                value='0|g')
+        self.assertReportedStat('zuul.pipeline.check.current_changes',
+                                value='0|g')
 
+    @skip("Disabled for early v3 development")
     def test_duplicate_pipelines(self):
         "Test that a change matching multiple pipelines works"
 
@@ -150,6 +112,7 @@
             self.assertNotIn('dup1/project-test1', A.messages[0])
             self.assertIn('dup2/project-test1', A.messages[0])
 
+    @skip("Disabled for early v3 development")
     def test_parallel_changes(self):
         "Test that changes are tested in parallel and merged in series"
 
@@ -227,6 +190,7 @@
         self.assertEqual(B.reported, 2)
         self.assertEqual(C.reported, 2)
 
+    @skip("Disabled for early v3 development")
     def test_failed_changes(self):
         "Test that a change behind a failed change is retested"
         self.worker.hold_jobs_in_build = True
@@ -257,6 +221,7 @@
         self.assertEqual(A.reported, 2)
         self.assertEqual(B.reported, 2)
 
+    @skip("Disabled for early v3 development")
     def test_independent_queues(self):
         "Test that changes end up in the right queues"
 
@@ -304,6 +269,7 @@
         self.assertEqual(B.reported, 2)
         self.assertEqual(C.reported, 2)
 
+    @skip("Disabled for early v3 development")
     def test_failed_change_at_head(self):
         "Test that if a change at the head fails, jobs behind it are canceled"
 
@@ -362,6 +328,7 @@
         self.assertEqual(B.reported, 2)
         self.assertEqual(C.reported, 2)
 
+    @skip("Disabled for early v3 development")
     def test_failed_change_in_middle(self):
         "Test a failed change in the middle of the queue"
 
@@ -438,6 +405,7 @@
         self.assertEqual(B.reported, 2)
         self.assertEqual(C.reported, 2)
 
+    @skip("Disabled for early v3 development")
     def test_failed_change_at_head_with_queue(self):
         "Test that if a change at the head fails, queued jobs are canceled"
 
@@ -500,6 +468,7 @@
         self.assertEqual(B.reported, 2)
         self.assertEqual(C.reported, 2)
 
+    @skip("Disabled for early v3 development")
     def _test_time_database(self, iteration):
         self.worker.hold_jobs_in_build = True
         A = self.fake_gerrit.addFakeChange('org/project', 'master', 'A')
@@ -534,12 +503,14 @@
         self.worker.release()
         self.waitUntilSettled()
 
+    @skip("Disabled for early v3 development")
     def test_time_database(self):
         "Test the time database"
 
         self._test_time_database(1)
         self._test_time_database(2)
 
+    @skip("Disabled for early v3 development")
     def test_two_failed_changes_at_head(self):
         "Test that changes are reparented correctly if 2 fail at head"
 
@@ -645,6 +616,7 @@
         self.assertEqual(B.reported, 2)
         self.assertEqual(C.reported, 2)
 
+    @skip("Disabled for early v3 development")
     def test_parse_skip_if(self):
         job_yaml = """
 jobs:
@@ -675,6 +647,7 @@
         matcher = self.sched._parseSkipIf(config_job)
         self.assertEqual(expected, matcher)
 
+    @skip("Disabled for early v3 development")
     def test_patch_order(self):
         "Test that dependent patches are tested in the right order"
         A = self.fake_gerrit.addFakeChange('org/project', 'master', 'A')
@@ -719,6 +692,7 @@
         self.assertEqual(B.reported, 2)
         self.assertEqual(C.reported, 2)
 
+    @skip("Disabled for early v3 development")
     def test_needed_changes_enqueue(self):
         "Test that a needed change is enqueued ahead"
         #          A      Given a git tree like this, if we enqueue
@@ -804,6 +778,7 @@
         self.assertEqual(self.history[6].changes,
                          '1,1 2,1 3,1 4,1 5,1 6,1 7,1')
 
+    @skip("Disabled for early v3 development")
     def test_source_cache(self):
         "Test that the source cache operates correctly"
         self.worker.hold_jobs_in_build = True
@@ -850,6 +825,7 @@
         self.assertEqual(A.queried, 2)  # Initial and isMerged
         self.assertEqual(B.queried, 3)  # Initial A, refresh from B, isMerged
 
+    @skip("Disabled for early v3 development")
     def test_can_merge(self):
         "Test whether a change is ready to merge"
         # TODO: move to test_gerrit (this is a unit test!)
@@ -867,6 +843,7 @@
         a = source._getChange(1, 2, refresh=True)
         self.assertTrue(source.canMerge(a, mgr.getSubmitAllowNeeds()))
 
+    @skip("Disabled for early v3 development")
     def test_build_configuration(self):
         "Test that zuul merges the right commits for testing"
 
@@ -901,6 +878,7 @@
         correct_messages = ['initial commit', 'A-1', 'B-1', 'C-1']
         self.assertEqual(repo_messages, correct_messages)
 
+    @skip("Disabled for early v3 development")
     def test_build_configuration_conflict(self):
         "Test that merge conflicts are handled"
 
@@ -946,6 +924,7 @@
         self.assertEqual(C.reported, 2)
         self.assertEqual(len(self.history), 6)
 
+    @skip("Disabled for early v3 development")
     def test_post(self):
         "Test that post jobs run"
 
@@ -968,6 +947,7 @@
         self.assertEqual(len(self.history), 1)
         self.assertIn('project-post', job_names)
 
+    @skip("Disabled for early v3 development")
     def test_post_ignore_deletes(self):
         "Test that deleting refs does not trigger post jobs"
 
@@ -990,6 +970,7 @@
         self.assertEqual(len(self.history), 0)
         self.assertNotIn('project-post', job_names)
 
+    @skip("Disabled for early v3 development")
     def test_post_ignore_deletes_negative(self):
         "Test that deleting refs does trigger post jobs"
 
@@ -1016,6 +997,7 @@
         self.assertEqual(len(self.history), 1)
         self.assertIn('project-post', job_names)
 
+    @skip("Disabled for early v3 development")
     def test_build_configuration_branch(self):
         "Test that the right commits are on alternate branches"
 
@@ -1050,6 +1032,7 @@
         correct_messages = ['initial commit', 'mp commit', 'A-1', 'B-1', 'C-1']
         self.assertEqual(repo_messages, correct_messages)
 
+    @skip("Disabled for early v3 development")
     def test_build_configuration_branch_interaction(self):
         "Test that switching between branches works"
         self.test_build_configuration()
@@ -1060,6 +1043,7 @@
         repo.heads.master.commit = repo.commit('init')
         self.test_build_configuration()
 
+    @skip("Disabled for early v3 development")
     def test_build_configuration_multi_branch(self):
         "Test that dependent changes on multiple branches are merged"
 
@@ -1142,6 +1126,7 @@
         self.assertNotEqual(ref_A, ref_B, ref_C)
         self.assertNotEqual(commit_A, commit_B, commit_C)
 
+    @skip("Disabled for early v3 development")
     def test_one_job_project(self):
         "Test that queueing works with one job"
         A = self.fake_gerrit.addFakeChange('org/one-job-project',
@@ -1159,6 +1144,7 @@
         self.assertEqual(B.data['status'], 'MERGED')
         self.assertEqual(B.reported, 2)
 
+    @skip("Disabled for early v3 development")
     def test_job_from_templates_launched(self):
         "Test whether a job generated via a template can be launched"
 
@@ -1172,6 +1158,7 @@
         self.assertEqual(self.getJobFromHistory('project-test2').result,
                          'SUCCESS')
 
+    @skip("Disabled for early v3 development")
     def test_layered_templates(self):
         "Test whether a job generated via a template can be launched"
 
@@ -1193,6 +1180,7 @@
         self.assertEqual(self.getJobFromHistory('project-test6').result,
                          'SUCCESS')
 
+    @skip("Disabled for early v3 development")
     def test_dependent_changes_dequeue(self):
         "Test that dependent patches are not needlessly tested"
 
@@ -1228,6 +1216,7 @@
         self.assertEqual(C.reported, 2)
         self.assertEqual(len(self.history), 1)
 
+    @skip("Disabled for early v3 development")
     def test_failing_dependent_changes(self):
         "Test that failing dependent patches are taken out of stream"
         self.worker.hold_jobs_in_build = True
@@ -1293,6 +1282,7 @@
         self.assertIn('Build succeeded', E.messages[1])
         self.assertEqual(len(self.history), 18)
 
+    @skip("Disabled for early v3 development")
     def test_head_is_dequeued_once(self):
         "Test that if a change at the head fails it is dequeued only once"
         # If it's dequeued more than once, we should see extra
@@ -1358,6 +1348,7 @@
         self.assertEqual(B.reported, 2)
         self.assertEqual(C.reported, 2)
 
+    @skip("Disabled for early v3 development")
     def test_nonvoting_job(self):
         "Test that non-voting jobs don't vote."
 
@@ -1384,6 +1375,7 @@
         for build in self.builds:
             self.assertEqual(build.parameters['ZUUL_VOTING'], '0')
 
+    @skip("Disabled for early v3 development")
     def test_check_queue_success(self):
         "Test successful check queue jobs."
 
@@ -1401,6 +1393,7 @@
         self.assertEqual(self.getJobFromHistory('project-test2').result,
                          'SUCCESS')
 
+    @skip("Disabled for early v3 development")
     def test_check_queue_failure(self):
         "Test failed check queue jobs."
 
@@ -1419,6 +1412,7 @@
         self.assertEqual(self.getJobFromHistory('project-test2').result,
                          'FAILURE')
 
+    @skip("Disabled for early v3 development")
     def test_dependent_behind_dequeue(self):
         "test that dependent changes behind dequeued changes work"
         # This complicated test is a reproduction of a real life bug
@@ -1505,6 +1499,7 @@
         self.assertEqual(self.countJobResults(self.history, 'ABORTED'), 15)
         self.assertEqual(len(self.history), 44)
 
+    @skip("Disabled for early v3 development")
     def test_merger_repack(self):
         "Test that the merger works after a repack"
 
@@ -1539,6 +1534,7 @@
         self.assertEqual(A.data['status'], 'MERGED')
         self.assertEqual(A.reported, 2)
 
+    @skip("Disabled for early v3 development")
     def test_merger_repack_large_change(self):
         "Test that the merger works with large changes after a repack"
         # https://bugs.launchpad.net/zuul/+bug/1078946
@@ -1565,6 +1561,7 @@
         self.assertEqual(A.data['status'], 'MERGED')
         self.assertEqual(A.reported, 2)
 
+    @skip("Disabled for early v3 development")
     def test_nonexistent_job(self):
         "Test launching a job that doesn't exist"
         # Set to the state immediately after a restart
@@ -1598,6 +1595,7 @@
         self.assertEqual(A.data['status'], 'MERGED')
         self.assertEqual(A.reported, 2)
 
+    @skip("Disabled for early v3 development")
     def test_single_nonexistent_post_job(self):
         "Test launching a single post job that doesn't exist"
         e = {
@@ -1621,6 +1619,7 @@
 
         self.assertEqual(len(self.history), 0)
 
+    @skip("Disabled for early v3 development")
     def test_new_patchset_dequeues_old(self):
         "Test that a new patchset causes the old to be dequeued"
         # D -> C (depends on B) -> B (depends on A) -> A -> M
@@ -1665,6 +1664,7 @@
         self.assertEqual(D.reported, 2)
         self.assertEqual(len(self.history), 9)  # 3 each for A, B, D.
 
+    @skip("Disabled for early v3 development")
     def test_new_patchset_check(self):
         "Test a new patchset in check"
 
@@ -1760,6 +1760,7 @@
         self.assertEqual(self.history[3].result, 'SUCCESS')
         self.assertEqual(self.history[3].changes, '1,1 2,2')
 
+    @skip("Disabled for early v3 development")
     def test_abandoned_gate(self):
         "Test that an abandoned change is dequeued from gate"
 
@@ -1785,6 +1786,7 @@
         self.assertEqual(A.reported, 1,
                          "Abandoned gate change should report only start")
 
+    @skip("Disabled for early v3 development")
     def test_abandoned_check(self):
         "Test that an abandoned change is dequeued from check"
 
@@ -1838,6 +1840,7 @@
         self.assertEqual(A.reported, 0, "Abandoned change should not report")
         self.assertEqual(B.reported, 1, "Change should report")
 
+    @skip("Disabled for early v3 development")
     def test_abandoned_not_timer(self):
         "Test that an abandoned change does not cancel timer jobs"
 
@@ -1873,6 +1876,7 @@
         self.worker.release()
         self.waitUntilSettled()
 
+    @skip("Disabled for early v3 development")
     def test_zuul_url_return(self):
         "Test if ZUUL_URL is returning when zuul_url is set in zuul.conf"
         self.assertTrue(self.sched.config.has_option('merger', 'zuul_url'))
@@ -1891,6 +1895,7 @@
         self.worker.release()
         self.waitUntilSettled()
 
+    @skip("Disabled for early v3 development")
     def test_new_patchset_dequeues_old_on_head(self):
         "Test that a new patchset causes the old to be dequeued (at head)"
         # D -> C (depends on B) -> B (depends on A) -> A -> M
@@ -1934,6 +1939,7 @@
         self.assertEqual(D.reported, 2)
         self.assertEqual(len(self.history), 7)
 
+    @skip("Disabled for early v3 development")
     def test_new_patchset_dequeues_old_without_dependents(self):
         "Test that a new patchset causes only the old to be dequeued"
         self.worker.hold_jobs_in_build = True
@@ -1965,6 +1971,7 @@
         self.assertEqual(C.reported, 2)
         self.assertEqual(len(self.history), 9)
 
+    @skip("Disabled for early v3 development")
     def test_new_patchset_dequeues_old_independent_queue(self):
         "Test that a new patchset causes the old to be dequeued (independent)"
         self.worker.hold_jobs_in_build = True
@@ -1993,6 +2000,7 @@
         self.assertEqual(len(self.history), 10)
         self.assertEqual(self.countJobResults(self.history, 'ABORTED'), 1)
 
+    @skip("Disabled for early v3 development")
     def test_noop_job(self):
         "Test that the internal noop job works"
         A = self.fake_gerrit.addFakeChange('org/noop-project', 'master', 'A')
@@ -2006,6 +2014,7 @@
         self.assertEqual(A.data['status'], 'MERGED')
         self.assertEqual(A.reported, 2)
 
+    @skip("Disabled for early v3 development")
     def test_no_job_project(self):
         "Test that reports with no jobs don't get sent"
         A = self.fake_gerrit.addFakeChange('org/no-jobs-project',
@@ -2023,6 +2032,7 @@
 
         self.assertEqual(len(self.history), 0)
 
+    @skip("Disabled for early v3 development")
     def test_zuul_refs(self):
         "Test that zuul refs exist and have the right changes"
         self.worker.hold_jobs_in_build = True
@@ -2110,6 +2120,7 @@
         self.assertEqual(D.data['status'], 'MERGED')
         self.assertEqual(D.reported, 2)
 
+    @skip("Disabled for early v3 development")
     def test_rerun_on_error(self):
         "Test that if a worker fails to run a job, it is run again"
         self.worker.hold_jobs_in_build = True
@@ -2125,6 +2136,7 @@
         self.assertEqual(self.countJobResults(self.history, 'RUN_ERROR'), 1)
         self.assertEqual(self.countJobResults(self.history, 'SUCCESS'), 3)
 
+    @skip("Disabled for early v3 development")
     def test_statsd(self):
         "Test each of the statsd methods used in the scheduler"
         import extras
@@ -2136,6 +2148,7 @@
         self.assertReportedStat('test-timing', '3|ms')
         self.assertReportedStat('test-gauge', '12|g')
 
+    @skip("Disabled for early v3 development")
     def test_stuck_job_cleanup(self):
         "Test that pending jobs are cleaned up if removed from layout"
         # This job won't be registered at startup because it is not in
@@ -2164,6 +2177,7 @@
         self.assertEqual(self.history[0].name, 'gate-noop')
         self.assertEqual(self.history[0].result, 'SUCCESS')
 
+    @skip("Disabled for early v3 development")
     def test_file_head(self):
         # This is a regression test for an observed bug.  A change
         # with a file named "HEAD" in the root directory of the repo
@@ -2188,6 +2202,7 @@
         self.assertIn('Build succeeded', A.messages[0])
         self.assertIn('Build succeeded', B.messages[0])
 
+    @skip("Disabled for early v3 development")
     def test_file_jobs(self):
         "Test that file jobs run only when appropriate"
         A = self.fake_gerrit.addFakeChange('org/project', 'master', 'A')
@@ -2209,6 +2224,7 @@
         self.assertEqual(B.data['status'], 'MERGED')
         self.assertEqual(B.reported, 2)
 
+    @skip("Disabled for early v3 development")
     def _test_skip_if_jobs(self, branch, should_skip):
         "Test that jobs with a skip-if filter run only when appropriate"
         self.updateConfigLayout(
@@ -2230,35 +2246,21 @@
         else:
             self.assertIn(change.data['number'], tested_change_ids)
 
+    @skip("Disabled for early v3 development")
     def test_skip_if_match_skips_job(self):
         self._test_skip_if_jobs(branch='master', should_skip=True)
 
+    @skip("Disabled for early v3 development")
     def test_skip_if_no_match_runs_job(self):
         self._test_skip_if_jobs(branch='mp', should_skip=False)
 
+    @skip("Disabled for early v3 development")
     def test_test_config(self):
         "Test that we can test the config"
         self.sched.testConfig(self.config.get('zuul', 'tenant_config'),
                               self.connections)
 
-    def test_build_description(self):
-        "Test that build descriptions update"
-        self.worker.registerFunction('set_description:' +
-                                     self.worker.worker_id)
-
-        A = self.fake_gerrit.addFakeChange('org/project', 'master', 'A')
-        A.addApproval('CRVW', 2)
-        self.fake_gerrit.addEvent(A.addApproval('APRV', 1))
-        self.waitUntilSettled()
-        desc = self.history[0].description
-        self.log.debug("Description: %s" % desc)
-        self.assertTrue(re.search("Branch.*master", desc))
-        self.assertTrue(re.search("Pipeline.*gate", desc))
-        self.assertTrue(re.search("project-merge.*SUCCESS", desc))
-        self.assertTrue(re.search("project-test1.*SUCCESS", desc))
-        self.assertTrue(re.search("project-test2.*SUCCESS", desc))
-        self.assertTrue(re.search("Reported result.*SUCCESS", desc))
-
+    @skip("Disabled for early v3 development")
     def test_queue_names(self):
         "Test shared change queue names"
         project1 = self.sched.layout.projects['org/project1']
@@ -2274,6 +2276,7 @@
             Exception, "More than one name assigned to change queue"):
             self.sched.reconfigure(self.config)
 
+    @skip("Disabled for early v3 development")
     def test_queue_precedence(self):
         "Test that queue precedence works"
 
@@ -2302,6 +2305,7 @@
         self.assertEqual(self.history[4].pipeline, 'check')
         self.assertEqual(self.history[5].pipeline, 'check')
 
+    @skip("Disabled for early v3 development")
     def test_json_status(self):
         "Test that we can retrieve JSON status info"
         self.worker.hold_jobs_in_build = True
@@ -2364,6 +2368,7 @@
         self.assertEqual('http://logs.example.com/1/1/gate/project-test2/2',
                          status_jobs[2]['report_url'])
 
+    @skip("Disabled for early v3 development")
     def test_merging_queues(self):
         "Test that transitively-connected change queues are merged"
         self.updateConfigLayout(
@@ -2371,6 +2376,7 @@
         self.sched.reconfigure(self.config)
         self.assertEqual(len(self.sched.layout.pipelines['gate'].queues), 1)
 
+    @skip("Disabled for early v3 development")
     def test_mutex(self):
         "Test job mutexes"
         self.config.set('zuul', 'layout_config',
@@ -2435,6 +2441,7 @@
         self.assertEqual(B.reported, 1)
         self.assertFalse('test-mutex' in self.sched.mutex.mutexes)
 
+    @skip("Disabled for early v3 development")
     def test_node_label(self):
         "Test that a job runs on a specific node label"
         self.worker.registerFunction('build:node-project-test1:debian')
@@ -2449,6 +2456,7 @@
                          'debian')
         self.assertIsNone(self.getJobFromHistory('node-project-test2').node)
 
+    @skip("Disabled for early v3 development")
     def test_live_reconfiguration(self):
         "Test that live reconfiguration works"
         self.worker.hold_jobs_in_build = True
@@ -2471,6 +2479,7 @@
         self.assertEqual(A.data['status'], 'MERGED')
         self.assertEqual(A.reported, 2)
 
+    @skip("Disabled for early v3 development")
     def test_live_reconfiguration_merge_conflict(self):
         # A real-world bug: a change in a gate queue has a merge
         # conflict and a job is added to its project while it's
@@ -2530,6 +2539,7 @@
                          'SUCCESS')
         self.assertEqual(len(self.history), 4)
 
+    @skip("Disabled for early v3 development")
     def test_live_reconfiguration_failed_root(self):
         # An extrapolation of test_live_reconfiguration_merge_conflict
         # that tests a job added to a job tree with a failed root does
@@ -2591,6 +2601,7 @@
         self.assertEqual(self.history[4].result, 'SUCCESS')
         self.assertEqual(len(self.history), 5)
 
+    @skip("Disabled for early v3 development")
     def test_live_reconfiguration_failed_job(self):
         # Test that a change with a removed failing job does not
         # disrupt reconfiguration.  If a change has a failed job and
@@ -2643,6 +2654,7 @@
         # Ensure the removed job was not included in the report.
         self.assertNotIn('project-test1', A.messages[0])
 
+    @skip("Disabled for early v3 development")
     def test_live_reconfiguration_shared_queue(self):
         # Test that a change with a failing job which was removed from
         # this project but otherwise still exists in the system does
@@ -2696,6 +2708,7 @@
         # Ensure the removed job was not included in the report.
         self.assertNotIn('project1-project2-integration', A.messages[0])
 
+    @skip("Disabled for early v3 development")
     def test_double_live_reconfiguration_shared_queue(self):
         # This was a real-world regression.  A change is added to
         # gate; a reconfigure happens, a second change which depends
@@ -2743,6 +2756,7 @@
         self.assertEqual(B.data['status'], 'MERGED')
         self.assertEqual(B.reported, 2)
 
+    @skip("Disabled for early v3 development")
     def test_live_reconfiguration_del_project(self):
         # Test project deletion from layout
         # while changes are enqueued
@@ -2793,6 +2807,7 @@
         self.assertEqual(len(self.sched.layout.pipelines['check'].queues), 0)
         self.assertIn('Build succeeded', A.messages[0])
 
+    @skip("Disabled for early v3 development")
     def test_live_reconfiguration_functions(self):
         "Test live reconfiguration with a custom function"
         self.worker.registerFunction('build:node-project-test1:debian')
@@ -2822,6 +2837,7 @@
                          'wheezy')
         self.assertIsNone(self.getJobFromHistory('node-project-test2').node)
 
+    @skip("Disabled for early v3 development")
     def test_delayed_repo_init(self):
         self.updateConfigLayout(
             'tests/fixtures/layout-delayed-repo-init.yaml')
@@ -2842,6 +2858,7 @@
         self.assertEqual(A.data['status'], 'MERGED')
         self.assertEqual(A.reported, 2)
 
+    @skip("Disabled for early v3 development")
     def test_repo_deleted(self):
         self.updateConfigLayout(
             'tests/fixtures/layout-repo-deleted.yaml')
@@ -2879,6 +2896,7 @@
         self.assertEqual(B.data['status'], 'MERGED')
         self.assertEqual(B.reported, 2)
 
+    @skip("Disabled for early v3 development")
     def test_tags(self):
         "Test job tags"
         self.config.set('zuul', 'layout_config',
@@ -2898,6 +2916,7 @@
             self.assertEqual(results.get(build.name, ''),
                              build.parameters.get('BUILD_TAGS'))
 
+    @skip("Disabled for early v3 development")
     def test_timer(self):
         "Test that a periodic job is triggered"
         self.worker.hold_jobs_in_build = True
@@ -2946,6 +2965,7 @@
         self.assertIn('project-bitrot-stable-old', status_jobs)
         self.assertIn('project-bitrot-stable-older', status_jobs)
 
+    @skip("Disabled for early v3 development")
     def test_idle(self):
         "Test that frequent periodic jobs work"
         self.worker.hold_jobs_in_build = True
@@ -2978,6 +2998,7 @@
             self.assertEqual(len(self.builds), 0)
             self.assertEqual(len(self.history), x * 2)
 
+    @skip("Disabled for early v3 development")
     def test_check_smtp_pool(self):
         self.updateConfigLayout(
             'tests/fixtures/layout-smtp.yaml')
@@ -3009,6 +3030,7 @@
         self.assertEqual(A.messages[0],
                          self.smtp_messages[1]['body'])
 
+    @skip("Disabled for early v3 development")
     def test_timer_smtp(self):
         "Test that a periodic job is triggered"
         self.worker.hold_jobs_in_build = True
@@ -3055,6 +3077,7 @@
         self.worker.release('.*')
         self.waitUntilSettled()
 
+    @skip("Disabled for early v3 development")
     def test_client_enqueue_change(self):
         "Test that the RPC client can enqueue a change"
         A = self.fake_gerrit.addFakeChange('org/project', 'master', 'A')
@@ -3078,6 +3101,7 @@
         self.assertEqual(A.reported, 2)
         self.assertEqual(r, True)
 
+    @skip("Disabled for early v3 development")
     def test_client_enqueue_ref(self):
         "Test that the RPC client can enqueue a ref"
 
@@ -3096,6 +3120,7 @@
         self.assertIn('project-post', job_names)
         self.assertEqual(r, True)
 
+    @skip("Disabled for early v3 development")
     def test_client_enqueue_negative(self):
         "Test that the RPC client returns errors"
         client = zuul.rpcclient.RPCClient('127.0.0.1',
@@ -3140,6 +3165,7 @@
         self.assertEqual(len(self.history), 0)
         self.assertEqual(len(self.builds), 0)
 
+    @skip("Disabled for early v3 development")
     def test_client_promote(self):
         "Test that the RPC client can promote a change"
         self.worker.hold_jobs_in_build = True
@@ -3213,6 +3239,7 @@
         client.shutdown()
         self.assertEqual(r, True)
 
+    @skip("Disabled for early v3 development")
     def test_client_promote_dependent(self):
         "Test that the RPC client can promote a dependent change"
         # C (depends on B) -> B -> A ; then promote C to get:
@@ -3280,6 +3307,7 @@
         client.shutdown()
         self.assertEqual(r, True)
 
+    @skip("Disabled for early v3 development")
     def test_client_promote_negative(self):
         "Test that the RPC client returns errors for promotion"
         self.worker.hold_jobs_in_build = True
@@ -3307,6 +3335,7 @@
         self.worker.release()
         self.waitUntilSettled()
 
+    @skip("Disabled for early v3 development")
     def test_queue_rate_limiting(self):
         "Test that DependentPipelines are rate limited with dep across window"
         self.updateConfigLayout(
@@ -3398,6 +3427,7 @@
         self.assertEqual(queue.window_floor, 1)
         self.assertEqual(C.data['status'], 'MERGED')
 
+    @skip("Disabled for early v3 development")
     def test_queue_rate_limiting_dependent(self):
         "Test that DependentPipelines are rate limited with dep in window"
         self.updateConfigLayout(
@@ -3471,6 +3501,7 @@
         self.assertEqual(queue.window_floor, 1)
         self.assertEqual(C.data['status'], 'MERGED')
 
+    @skip("Disabled for early v3 development")
     def test_worker_update_metadata(self):
         "Test if a worker can send back metadata about itself"
         self.worker.hold_jobs_in_build = True
@@ -3509,6 +3540,7 @@
         self.worker.release()
         self.waitUntilSettled()
 
+    @skip("Disabled for early v3 development")
     def test_footer_message(self):
         "Test a pipeline's footer message is correctly added to the report."
         self.updateConfigLayout(
@@ -3549,6 +3581,7 @@
         self.assertEqual(failure_body, self.smtp_messages[0]['body'])
         self.assertEqual(success_body, self.smtp_messages[1]['body'])
 
+    @skip("Disabled for early v3 development")
     def test_merge_failure_reporters(self):
         """Check that the config is set up correctly"""
 
@@ -3594,6 +3627,7 @@
             )
         )
 
+    @skip("Disabled for early v3 development")
     def test_merge_failure_reports(self):
         """Check that when a change fails to merge the correct message is sent
         to the correct reporter"""
@@ -3629,6 +3663,7 @@
         self.assertEqual('The merge failed! For more information...',
                          self.smtp_messages[0]['body'])
 
+    @skip("Disabled for early v3 development")
     def test_default_merge_failure_reports(self):
         """Check that the default merge failure reports are correct."""
 
@@ -3654,6 +3689,7 @@
         self.assertNotIn('logs.example.com', B.messages[1])
         self.assertNotIn('SKIPPED', B.messages[1])
 
+    @skip("Disabled for early v3 development")
     def test_swift_instructions(self):
         "Test that the correct swift instructions are sent to the workers"
         self.updateConfigLayout(
@@ -3699,6 +3735,7 @@
         self.worker.release()
         self.waitUntilSettled()
 
+    @skip("Disabled for early v3 development")
     def test_client_get_running_jobs(self):
         "Test that the RPC client can get a list of running jobs"
         self.worker.hold_jobs_in_build = True
@@ -3757,6 +3794,7 @@
         running_items = client.get_running_jobs()
         self.assertEqual(0, len(running_items))
 
+    @skip("Disabled for early v3 development")
     def test_nonvoting_pipeline(self):
         "Test that a nonvoting pipeline (experimental) can still report"
 
@@ -3769,6 +3807,7 @@
             'SUCCESS')
         self.assertEqual(A.reported, 1)
 
+    @skip("Disabled for early v3 development")
     def test_crd_gate(self):
         "Test cross-repo dependencies"
         A = self.fake_gerrit.addFakeChange('org/project1', 'master', 'A')
@@ -3834,6 +3873,7 @@
         self.assertEqual(self.getJobFromHistory('project1-merge').changes,
                          '2,1 1,1')
 
+    @skip("Disabled for early v3 development")
     def test_crd_branch(self):
         "Test cross-repo dependencies in multiple branches"
         A = self.fake_gerrit.addFakeChange('org/project1', 'master', 'A')
@@ -3874,6 +3914,7 @@
         self.assertEqual(self.getJobFromHistory('project1-merge').changes,
                          '2,1 3,1 1,1')
 
+    @skip("Disabled for early v3 development")
     def test_crd_multiline(self):
         "Test multiple depends-on lines in commit"
         A = self.fake_gerrit.addFakeChange('org/project1', 'master', 'A')
@@ -3913,6 +3954,7 @@
         self.assertEqual(self.getJobFromHistory('project1-merge').changes,
                          '2,1 3,1 1,1')
 
+    @skip("Disabled for early v3 development")
     def test_crd_unshared_gate(self):
         "Test cross-repo dependencies in unshared gate queues"
         A = self.fake_gerrit.addFakeChange('org/project1', 'master', 'A')
@@ -3951,6 +3993,7 @@
         self.assertEqual(A.data['status'], 'MERGED')
         self.assertEqual(A.reported, 2)
 
+    @skip("Disabled for early v3 development")
     def test_crd_gate_reverse(self):
         "Test reverse cross-repo dependencies"
         A = self.fake_gerrit.addFakeChange('org/project1', 'master', 'A')
@@ -3990,6 +4033,7 @@
         self.assertEqual(self.getJobFromHistory('project1-merge').changes,
                          '2,1 1,1')
 
+    @skip("Disabled for early v3 development")
     def test_crd_cycle(self):
         "Test cross-repo dependency cycles"
         A = self.fake_gerrit.addFakeChange('org/project1', 'master', 'A')
@@ -4012,6 +4056,7 @@
         self.assertEqual(A.data['status'], 'NEW')
         self.assertEqual(B.data['status'], 'NEW')
 
+    @skip("Disabled for early v3 development")
     def test_crd_gate_unknown(self):
         "Test unknown projects in dependent pipeline"
         self.init_repo("org/unknown")
@@ -4054,6 +4099,7 @@
         self.assertEqual(B.data['status'], 'MERGED')
         self.assertEqual(B.reported, 0)
 
+    @skip("Disabled for early v3 development")
     def test_crd_check(self):
         "Test cross-repo dependencies in independent pipelines"
 
@@ -4096,6 +4142,7 @@
         self.assertEqual(self.history[0].changes, '2,1 1,1')
         self.assertEqual(len(self.sched.layout.pipelines['check'].queues), 0)
 
+    @skip("Disabled for early v3 development")
     def test_crd_check_git_depends(self):
         "Test single-repo dependencies in independent pipelines"
         self.gearman_server.hold_jobs_in_build = True
@@ -4126,6 +4173,7 @@
         self.assertIn('Build succeeded', A.messages[0])
         self.assertIn('Build succeeded', B.messages[0])
 
+    @skip("Disabled for early v3 development")
     def test_crd_check_duplicate(self):
         "Test duplicate check in independent pipelines"
         self.worker.hold_jobs_in_build = True
@@ -4168,6 +4216,7 @@
         self.assertIn('Build succeeded', A.messages[0])
         self.assertIn('Build succeeded', B.messages[0])
 
+    @skip("Disabled for early v3 development")
     def _test_crd_check_reconfiguration(self, project1, project2):
         "Test cross-repo dependencies re-enqueued in independent pipelines"
 
@@ -4206,9 +4255,11 @@
         self.assertEqual(self.history[0].changes, '2,1 1,1')
         self.assertEqual(len(self.sched.layout.pipelines['check'].queues), 0)
 
+    @skip("Disabled for early v3 development")
     def test_crd_check_reconfiguration(self):
         self._test_crd_check_reconfiguration('org/project1', 'org/project2')
 
+    @skip("Disabled for early v3 development")
     def test_crd_undefined_project(self):
         """Test that undefined projects in dependencies are handled for
         independent pipelines"""
@@ -4217,6 +4268,7 @@
         self.init_repo("org/unknown")
         self._test_crd_check_reconfiguration('org/project1', 'org/unknown')
 
+    @skip("Disabled for early v3 development")
     def test_crd_check_ignore_dependencies(self):
         "Test cross-repo dependencies can be ignored"
         self.updateConfigLayout(
@@ -4262,6 +4314,7 @@
         for job in self.history:
             self.assertEqual(len(job.changes.split()), 1)
 
+    @skip("Disabled for early v3 development")
     def test_crd_check_transitive(self):
         "Test transitive cross-repo dependencies"
         # Specifically, if A -> B -> C, and C gets a new patchset and
@@ -4302,6 +4355,7 @@
         self.waitUntilSettled()
         self.assertEqual(self.history[-1].changes, '3,2 2,1 1,2')
 
+    @skip("Disabled for early v3 development")
     def test_crd_cycle_join(self):
         "Test an updated change creates a cycle"
         A = self.fake_gerrit.addFakeChange('org/project2', 'master', 'A')
@@ -4341,6 +4395,7 @@
         source._getChange(u'1', u'2', True)
         source._getChange(u'2', u'2', True)
 
+    @skip("Disabled for early v3 development")
     def test_disable_at(self):
         "Test a pipeline will only report to the disabled trigger when failing"
 
@@ -4463,6 +4518,7 @@
         # No more messages reported via smtp
         self.assertEqual(3, len(self.smtp_messages))
 
+    @skip("Disabled for early v3 development")
     def test_success_pattern(self):
         "Ensure bad build params are ignored"
 
diff --git a/tests/test_v3.py b/tests/test_v3.py
index 50e20c8..f3064e5 100644
--- a/tests/test_v3.py
+++ b/tests/test_v3.py
@@ -17,16 +17,14 @@
 import logging
 import textwrap
 
-from tests.base import (
-    ZuulTestCase,
-)
+from tests.base import AnsibleZuulTestCase
 
 logging.basicConfig(level=logging.DEBUG,
                     format='%(asctime)s %(name)-32s '
                     '%(levelname)-8s %(message)s')
 
 
-class TestMultipleTenants(ZuulTestCase):
+class TestMultipleTenants(AnsibleZuulTestCase):
     # A temporary class to hold new tests while others are disabled
 
     tenant_config_file = 'config/multi-tenant/main.yaml'
@@ -69,7 +67,7 @@
                          "not affect tenant one")
 
 
-class TestInRepoConfig(ZuulTestCase):
+class TestInRepoConfig(AnsibleZuulTestCase):
     # A temporary class to hold new tests while others are disabled
 
     tenant_config_file = 'config/in-repo/main.yaml'
@@ -114,7 +112,7 @@
                       "A should transit tenant-one gate")
 
 
-class TestProjectTemplate(ZuulTestCase):
+class TestProjectTemplate(AnsibleZuulTestCase):
     tenant_config_file = 'config/project-template/main.yaml'
 
     def test(self):
diff --git a/zuul/configloader.py b/zuul/configloader.py
index 70a880d..65ec803 100644
--- a/zuul/configloader.py
+++ b/zuul/configloader.py
@@ -40,17 +40,23 @@
 class JobParser(object):
     @staticmethod
     def getSchema():
-        # TODOv3(jeblair, jhesketh): move to auth
-        swift = {vs.Required('name'): str,
-                 'container': str,
-                 'expiry': int,
-                 'max_file_size': int,
-                 'max-file-size': int,
-                 'max_file_count': int,
-                 'max-file-count': int,
-                 'logserver_prefix': str,
-                 'logserver-prefix': str,
-                 }
+        swift_tmpurl = {vs.Required('name'): str,
+                        'container': str,
+                        'expiry': int,
+                        'max_file_size': int,
+                        'max-file-size': int,
+                        'max_file_count': int,
+                        'max-file-count': int,
+                        'logserver_prefix': str,
+                        'logserver-prefix': str,
+                        }
+
+        password = {str: str}
+
+        auth = {'password': to_list(password),
+                'inherit': bool,
+                'swift-tmpurl': to_list(swift_tmpurl),
+                }
 
         node = {vs.Required('name'): str,
                 vs.Required('image'): str,
@@ -68,7 +74,7 @@
                'tags': to_list(str),
                'branches': to_list(str),
                'files': to_list(str),
-               'swift': to_list(swift),
+               'auth': to_list(auth),
                'irrelevant-files': to_list(str),
                'nodes': [node],
                'timeout': int,
@@ -81,6 +87,8 @@
     def fromYaml(layout, conf):
         JobParser.getSchema()(conf)
         job = model.Job(conf['name'])
+        if 'auth' in conf:
+            job.auth = conf.get('auth')
         if 'parent' in conf:
             parent = layout.getJob(conf['parent'])
             job.inheritFrom(parent)
diff --git a/zuul/launcher/client.py b/zuul/launcher/client.py
index 8448422..14773cb 100644
--- a/zuul/launcher/client.py
+++ b/zuul/launcher/client.py
@@ -194,11 +194,6 @@
             port = config.get('gearman', 'port')
         else:
             port = 4730
-        if config.has_option('gearman', 'check_job_registration'):
-            self.job_registration = config.getboolean(
-                'gearman', 'check_job_registration')
-        else:
-            self.job_registration = True
 
         self.gearman = ZuulGearmanClient(self)
         self.gearman.addServer(server, port)
@@ -259,7 +254,7 @@
         # NOTE(jhesketh): The params need to stay in a key=value data pair
         # as workers cannot necessarily handle lists.
 
-        if job.swift and self.swift.connection:
+        if 'swift' in job.auth and self.swift.connection:
 
             for name, s in job.swift.items():
                 swift_instructions = {}
@@ -397,13 +392,6 @@
         build.__gearman_job = gearman_job
         self.builds[uuid] = build
 
-        if self.job_registration and not self.isJobRegistered(
-                gearman_job.name):
-            self.log.error("Job %s is not registered with Gearman" %
-                           gearman_job)
-            self.onBuildCompleted(gearman_job, 'NOT_REGISTERED')
-            return build
-
         if pipeline.precedence == zuul.model.PRECEDENCE_NORMAL:
             precedence = gear.PRECEDENCE_NORMAL
         elif pipeline.precedence == zuul.model.PRECEDENCE_HIGH:
@@ -541,28 +529,6 @@
                                timeout=300)
         return True
 
-    def setBuildDescription(self, build, desc):
-        try:
-            name = "set_description:%s" % build.__gearman_manager
-        except AttributeError:
-            # We haven't yet received the first data packet that tells
-            # us where the job is running.
-            return False
-
-        if self.job_registration and not self.isJobRegistered(name):
-            return False
-
-        desc_uuid = str(uuid4().hex)
-        data = dict(name=build.job.name,
-                    number=build.number,
-                    html_description=desc)
-        desc_job = gear.Job(name, json.dumps(data), unique=desc_uuid)
-        self.meta_jobs[desc_uuid] = desc_job
-        self.log.debug("Submitting describe job: %s", desc_job)
-        self.gearman.submitJob(desc_job, precedence=gear.PRECEDENCE_LOW,
-                               timeout=300)
-        return True
-
     def lookForLostBuilds(self):
         self.log.debug("Looking for lost builds")
         for build in self.builds.values():
diff --git a/zuul/launcher/server.py b/zuul/launcher/server.py
index 89969cd..45c42dd 100644
--- a/zuul/launcher/server.py
+++ b/zuul/launcher/server.py
@@ -28,6 +28,11 @@
 import zuul.merger
 
 
+# TODOv3(mordred): put git repos in a hierarchy that includes source
+# hostname, eg: git.openstack.org/openstack/nova.  Also, configure
+# sources to have an alias, so that the review.openstack.org source
+# repos end up in git.openstack.org.
+
 class JobDir(object):
     def __init__(self):
         self.root = tempfile.mkdtemp()
@@ -238,7 +243,7 @@
 
             # TODOv3: Ansible the ansible thing here.
             self.prepareAnsibleFiles(jobdir, args)
-            result = self.runAnsible(jobdir)
+            result = self.runAnsible(jobdir, job)
 
             data = {
                 'url': 'https://server/job',
@@ -272,7 +277,8 @@
             config.write('[defaults]\n')
             config.write('hostfile = %s\n' % jobdir.inventory)
 
-    def runAnsible(self, jobdir):
+    def runAnsible(self, jobdir, job):
+        # Job is included here for the benefit of the test framework.
         proc = subprocess.Popen(
             ['ansible-playbook', jobdir.playbook],
             cwd=jobdir.ansible_root,
diff --git a/zuul/manager/__init__.py b/zuul/manager/__init__.py
index 4a4fa0a..c136343 100644
--- a/zuul/manager/__init__.py
+++ b/zuul/manager/__init__.py
@@ -10,14 +10,11 @@
 # License for the specific language governing permissions and limitations
 # under the License.
 
-import extras
 import logging
 
 from zuul import exceptions
 from zuul.model import NullChange
 
-statsd = extras.try_import('statsd.statsd')
-
 
 class DynamicChangeQueueContextManager(object):
     def __init__(self, change_queue):
@@ -404,7 +401,6 @@
                                    "for change %s" % (build, item.change))
             build.result = 'CANCELED'
             canceled = True
-        self.updateBuildDescriptions(old_build_set)
         for item_behind in item.items_behind:
             self.log.debug("Canceling jobs for change %s, behind change %s" %
                            (item_behind.change, item.change))
@@ -587,26 +583,6 @@
                        (self.pipeline.name, changed))
         return changed
 
-    def updateBuildDescriptions(self, build_set):
-        for build in build_set.getBuilds():
-            try:
-                desc = self.formatDescription(build)
-                self.sched.launcher.setBuildDescription(build, desc)
-            except:
-                # Log the failure and let loop continue
-                self.log.error("Failed to update description for build %s" %
-                               (build))
-
-        if build_set.previous_build_set:
-            for build in build_set.previous_build_set.getBuilds():
-                try:
-                    desc = self.formatDescription(build)
-                    self.sched.launcher.setBuildDescription(build, desc)
-                except:
-                    # Log the failure and let loop continue
-                    self.log.error("Failed to update description for "
-                                   "build %s in previous build set" % (build))
-
     def onBuildStarted(self, build):
         self.log.debug("Build %s started" % build)
         return True
@@ -713,113 +689,10 @@
             except:
                 self.log.exception("Exception while reporting:")
                 item.setReportedResult('ERROR')
-        self.updateBuildDescriptions(item.current_build_set)
-        return ret
-
-    def formatDescription(self, build):
-        concurrent_changes = ''
-        concurrent_builds = ''
-        other_builds = ''
-
-        for change in build.build_set.other_changes:
-            concurrent_changes += '<li><a href="{change.url}">\
-              {change.number},{change.patchset}</a></li>'.format(
-                change=change)
-
-        change = build.build_set.item.change
-
-        for build in build.build_set.getBuilds():
-            if build.url:
-                concurrent_builds += """\
-<li>
-  <a href="{build.url}">
-  {build.job.name} #{build.number}</a>: {build.result}
-</li>
-""".format(build=build)
-            else:
-                concurrent_builds += """\
-<li>
-  {build.job.name}: {build.result}
-</li>""".format(build=build)
-
-        if build.build_set.previous_build_set:
-            other_build = build.build_set.previous_build_set.getBuild(
-                build.job.name)
-            if other_build:
-                other_builds += """\
-<li>
-  Preceded by: <a href="{build.url}">
-  {build.job.name} #{build.number}</a>
-</li>
-""".format(build=other_build)
-
-        if build.build_set.next_build_set:
-            other_build = build.build_set.next_build_set.getBuild(
-                build.job.name)
-            if other_build:
-                other_builds += """\
-<li>
-  Succeeded by: <a href="{build.url}">
-  {build.job.name} #{build.number}</a>
-</li>
-""".format(build=other_build)
-
-        result = build.build_set.result
-
-        if hasattr(change, 'number'):
-            ret = """\
-<p>
-  Triggered by change:
-    <a href="{change.url}">{change.number},{change.patchset}</a><br/>
-  Branch: <b>{change.branch}</b><br/>
-  Pipeline: <b>{self.pipeline.name}</b>
-</p>"""
-        elif hasattr(change, 'ref'):
-            ret = """\
-<p>
-  Triggered by reference:
-    {change.ref}</a><br/>
-  Old revision: <b>{change.oldrev}</b><br/>
-  New revision: <b>{change.newrev}</b><br/>
-  Pipeline: <b>{self.pipeline.name}</b>
-</p>"""
-        else:
-            ret = ""
-
-        if concurrent_changes:
-            ret += """\
-<p>
-  Other changes tested concurrently with this change:
-  <ul>{concurrent_changes}</ul>
-</p>
-"""
-        if concurrent_builds:
-            ret += """\
-<p>
-  All builds for this change set:
-  <ul>{concurrent_builds}</ul>
-</p>
-"""
-
-        if other_builds:
-            ret += """\
-<p>
-  Other build sets for this change:
-  <ul>{other_builds}</ul>
-</p>
-"""
-        if result:
-            ret += """\
-<p>
-  Reported result: <b>{result}</b>
-</p>
-"""
-
-        ret = ret.format(**locals())
         return ret
 
     def reportStats(self, item):
-        if not statsd:
+        if not self.sched.statsd:
             return
         try:
             # Update the gauge on enqueue and dequeue, but timers only
@@ -834,17 +707,17 @@
             # stats_counts.zuul.pipeline.NAME.total_changes
             # stats.gauges.zuul.pipeline.NAME.current_changes
             key = 'zuul.pipeline.%s' % self.pipeline.name
-            statsd.gauge(key + '.current_changes', items)
+            self.sched.statsd.gauge(key + '.current_changes', items)
             if dt:
-                statsd.timing(key + '.resident_time', dt)
-                statsd.incr(key + '.total_changes')
+                self.sched.statsd.timing(key + '.resident_time', dt)
+                self.sched.statsd.incr(key + '.total_changes')
 
             # stats.timers.zuul.pipeline.NAME.ORG.PROJECT.resident_time
             # stats_counts.zuul.pipeline.NAME.ORG.PROJECT.total_changes
             project_name = item.change.project.name.replace('/', '.')
             key += '.%s' % project_name
             if dt:
-                statsd.timing(key + '.resident_time', dt)
-                statsd.incr(key + '.total_changes')
+                self.sched.statsd.timing(key + '.resident_time', dt)
+                self.sched.statsd.incr(key + '.total_changes')
         except:
             self.log.exception("Exception reporting pipeline stats")
diff --git a/zuul/model.py b/zuul/model.py
index 2f5f001..2b957a5 100644
--- a/zuul/model.py
+++ b/zuul/model.py
@@ -371,7 +371,7 @@
         timeout=None,
         # variables={},
         nodes=[],
-        # auth={},
+        auth={},
         workspace=None,
         pre_run=None,
         post_run=None,
@@ -385,7 +385,6 @@
         branch_matcher=None,
         file_matcher=None,
         irrelevant_file_matcher=None,  # skip-if
-        swift=None,  # TODOv3(jeblair): move to auth
         parameter_function=None,  # TODOv3(jeblair): remove
         success_pattern=None,  # TODOv3(jeblair): remove
         tags=set(),
@@ -424,8 +423,11 @@
         if not isinstance(other, Job):
             raise Exception("Job unable to inherit from %s" % (other,))
         for k, v in self.attributes.items():
-            if getattr(other, k) != v:
+            if getattr(other, k) != v and k != 'auth':
                 setattr(self, k, getattr(other, k))
+        # Inherit auth only if explicitly allowed
+        if other.auth and 'inherit' in other.auth and other.auth['inherit']:
+            setattr(self, 'auth', getattr(other, 'auth'))
 
     def changeMatches(self, change):
         if self.branch_matcher and not self.branch_matcher.matches(change):
@@ -1163,6 +1165,8 @@
 
 
 class NullChange(Changeish):
+    # TODOv3(jeblair): remove this in favor of enqueueing Refs (eg
+    # current master) instead.
     def __repr__(self):
         return '<NullChange for %s>' % (self.project)
 
diff --git a/zuul/scheduler.py b/zuul/scheduler.py
index 516be80..8b7f50f 100644
--- a/zuul/scheduler.py
+++ b/zuul/scheduler.py
@@ -31,8 +31,6 @@
 from zuul import exceptions
 from zuul import version as zuul_version
 
-statsd = extras.try_import('statsd.statsd')
-
 
 class MutexHandler(object):
     log = logging.getLogger("zuul.MutexHandler")
@@ -234,6 +232,7 @@
         self.launcher = None
         self.merger = None
         self.connections = None
+        self.statsd = extras.try_import('statsd.statsd')
         # TODO(jeblair): fix this
         self.mutex = MutexHandler()
         # Despite triggers being part of the pipeline, there is one trigger set
@@ -287,8 +286,8 @@
     def addEvent(self, event):
         self.log.debug("Adding trigger event: %s" % event)
         try:
-            if statsd:
-                statsd.incr('gerrit.event.%s' % event.type)
+            if self.statsd:
+                self.statsd.incr('gerrit.event.%s' % event.type)
         except:
             self.log.exception("Exception reporting event stats")
         self.trigger_event_queue.put(event)
@@ -313,10 +312,10 @@
         # timing) is recorded before setting the result.
         build.result = result
         try:
-            if statsd and build.pipeline:
+            if self.statsd and build.pipeline:
                 jobname = build.job.name.replace('.', '_')
                 key = 'zuul.pipeline.%s.all_jobs' % build.pipeline.name
-                statsd.incr(key)
+                self.statsd.incr(key)
                 for label in build.node_labels:
                     # Jenkins includes the node name in its list of labels, so
                     # we filter it out here, since that is not statistically
@@ -326,18 +325,18 @@
                     dt = int((build.start_time - build.launch_time) * 1000)
                     key = 'zuul.pipeline.%s.label.%s.wait_time' % (
                         build.pipeline.name, label)
-                    statsd.timing(key, dt)
+                    self.statsd.timing(key, dt)
                 key = 'zuul.pipeline.%s.job.%s.%s' % (build.pipeline.name,
                                                       jobname, build.result)
                 if build.result in ['SUCCESS', 'FAILURE'] and build.start_time:
                     dt = int((build.end_time - build.start_time) * 1000)
-                    statsd.timing(key, dt)
-                statsd.incr(key)
+                    self.statsd.timing(key, dt)
+                self.statsd.incr(key)
 
                 key = 'zuul.pipeline.%s.job.%s.wait_time' % (
                     build.pipeline.name, jobname)
                 dt = int((build.start_time - build.launch_time) * 1000)
-                statsd.timing(key, dt)
+                self.statsd.timing(key, dt)
         except:
             self.log.exception("Exception reporting runtime stats")
         event = BuildCompletedEvent(build)
@@ -475,12 +474,7 @@
         finally:
             self.layout_lock.release()
 
-    def _reconfigureTenant(self, tenant):
-        # This is called from _doReconfigureEvent while holding the
-        # layout lock
-        old_tenant = self.abide.tenants.get(tenant.name)
-        if not old_tenant:
-            return
+    def _reenqueueTenant(self, old_tenant, tenant):
         for name, new_pipeline in tenant.layout.pipelines.items():
             old_pipeline = old_tenant.layout.pipelines.get(name)
             if not old_pipeline:
@@ -526,20 +520,28 @@
                     self.log.exception(
                         "Exception while canceling build %s "
                         "for change %s" % (build, item.change))
+
+    def _reconfigureTenant(self, tenant):
+        # This is called from _doReconfigureEvent while holding the
+        # layout lock
+        old_tenant = self.abide.tenants.get(tenant.name)
+        if old_tenant:
+            self._reenqueueTenant(old_tenant, tenant)
         # TODOv3(jeblair): update for tenants
-        self.maintainConnectionCache()
+        # self.maintainConnectionCache()
         for pipeline in tenant.layout.pipelines.values():
             pipeline.source.postConfig()
-            pipeline.trigger.postConfig(pipeline)
+            for trigger in pipeline.triggers:
+                trigger.postConfig(pipeline)
             for reporter in pipeline.actions:
                 reporter.postConfig()
-        if statsd:
+        if self.statsd:
             try:
-                for pipeline in self.layout.pipelines.values():
+                for pipeline in tenant.layout.pipelines.values():
                     items = len(pipeline.getAllItems())
                     # stats.gauges.zuul.pipeline.NAME.current_changes
                     key = 'zuul.pipeline.%s' % pipeline.name
-                    statsd.gauge(key + '.current_changes', items)
+                    self.statsd.gauge(key + '.current_changes', items)
             except Exception:
                 self.log.exception("Exception reporting initial "
                                    "pipeline stats:")
@@ -612,7 +614,7 @@
         return False
 
     def run(self):
-        if statsd:
+        if self.statsd:
             self.log.debug("Statsd enabled")
         else:
             self.log.debug("Statsd disabled because python statsd "