Merge "Add zuul.d configuration split documentation" into feature/zuulv3
diff --git a/doc/source/user/config.rst b/doc/source/user/config.rst
index bfb8136..4a9a99e 100644
--- a/doc/source/user/config.rst
+++ b/doc/source/user/config.rst
@@ -718,8 +718,23 @@
   roles: a Galaxy role, which is simply a role that is installed from
   Ansible Galaxy, or a Zuul role, which is a role provided by a
   project managed by Zuul.  Zuul roles are able to benefit from
-  speculative merging and cross-project dependencies when used by jobs
-  in untrusted projects.
+  speculative merging and cross-project dependencies when used by
+  playbooks in untrusted projects.  Roles are added to the Ansible
+  role path in the order they appear on the job -- roles earlier in
+  the list will take precedence over those which follow.
+
+  In the case of job inheritance or variance, the roles used for each
+  of the playbooks run by the job will be only those which were
+  defined along with that playbook.  If a child job inherits from a
+  parent which defines a pre and post playbook, then the pre and post
+  playbooks it inherits from the parent job will run only with the
+  roles that were defined on the parent.  If the child adds its own
+  pre and post playbooks, then any roles added by the child will be
+  available to the child's playbooks.  This is so that a job which
+  inherits from a parent does not inadvertantly alter the behavior of
+  the parent's playbooks by the addition of conflicting roles.  Roles
+  added by a child will appear before those it inherits from its
+  parent.
 
   A project which supplies a role may be structured in one of two
   configurations: a bare role (in which the role exists at the root of
diff --git a/doc/source/user/jobs.rst b/doc/source/user/jobs.rst
index 78121bc..c2c376e 100644
--- a/doc/source/user/jobs.rst
+++ b/doc/source/user/jobs.rst
@@ -70,8 +70,41 @@
 
 .. TODO: link to base job documentation and/or document src (and logs?) directory
 
+Variables
+---------
+
+Any variables specified in the job definition are available as Ansible
+host variables.  They are added to the `vars` section of the inventory
+file under the `all` hosts group, so they are available to all hosts.
+Simply refer to them by the name specified in the job's `vars`
+section.
+
+Secrets
+~~~~~~~
+
+Secrets also appear as variables available to Ansible.  Unlike job
+variables, these are not added to the inventory file (so that the
+inventory file may be kept for debugging purposes without revealing
+secrets).  But they are still available to Ansible as normal
+variables.  Because secrets are groups of variables, they will appear
+as a dictionary structure in templates, with the dictionary itself
+being the name of the secret, and its members the individual items in
+the secret.  For example, a secret defined as::
+
+  - secret:
+      name: credentials
+      data:
+        username: foo
+        password: bar
+
+Might be used in a template as::
+
+ {{ credentials.username }} {{ credentials.password }}
+
+.. TODO: xref job vars
+
 Zuul Variables
---------------
+~~~~~~~~~~~~~~
 
 Zuul supplies not only the variables specified by the job definition
 to Ansible, but also some variables from the executor itself.  They
diff --git a/tests/base.py b/tests/base.py
index 9709bf7..fb94638 100755
--- a/tests/base.py
+++ b/tests/base.py
@@ -1308,7 +1308,7 @@
         self.running_builds.append(build)
         self.job_builds[job.unique] = build
         args = json.loads(job.arguments)
-        args['vars']['zuul']['_test'] = dict(test_root=self._test_root)
+        args['zuul']['_test'] = dict(test_root=self._test_root)
         job.arguments = json.dumps(args)
         self.job_workers[job.unique] = RecordingAnsibleJob(self, job)
         self.job_workers[job.unique].run()
@@ -1359,12 +1359,12 @@
         self.recordResult(result)
         return result
 
-    def runAnsible(self, cmd, timeout, trusted=False):
+    def runAnsible(self, cmd, timeout, config_file, trusted):
         build = self.executor_server.job_builds[self.job.unique]
 
         if self.executor_server._run_ansible:
             result = super(RecordingAnsibleJob, self).runAnsible(
-                cmd, timeout, trusted=trusted)
+                cmd, timeout, config_file, trusted)
         else:
             result = build.run()
         return result
diff --git a/tests/fixtures/config/in-repo/git/common-config/zuul.yaml b/tests/fixtures/config/in-repo/git/common-config/zuul.yaml
index 1fdaf2e..fce086e 100644
--- a/tests/fixtures/config/in-repo/git/common-config/zuul.yaml
+++ b/tests/fixtures/config/in-repo/git/common-config/zuul.yaml
@@ -32,6 +32,26 @@
         verified: 0
     precedence: high
 
+- pipeline:
+    name: gate
+    manager: dependent
+    trigger:
+      gerrit:
+        - event: comment-added
+          approval:
+            - code-review: 2
+    success:
+      gerrit:
+        verified: 2
+        submit: true
+    failure:
+      gerrit:
+        verified: -2
+    start:
+      gerrit:
+        verified: 0
+    precedence: high
+
 - job:
     name: common-config-test
 
diff --git a/tests/fixtures/config/roles/git/org_project/playbooks/parent-post.yaml b/tests/fixtures/config/roles/git/org_project/playbooks/parent-post.yaml
new file mode 100644
index 0000000..f679dce
--- /dev/null
+++ b/tests/fixtures/config/roles/git/org_project/playbooks/parent-post.yaml
@@ -0,0 +1,2 @@
+- hosts: all
+  tasks: []
diff --git a/tests/fixtures/config/roles/git/org_project/playbooks/parent-pre.yaml b/tests/fixtures/config/roles/git/org_project/playbooks/parent-pre.yaml
new file mode 100644
index 0000000..f679dce
--- /dev/null
+++ b/tests/fixtures/config/roles/git/org_project/playbooks/parent-pre.yaml
@@ -0,0 +1,2 @@
+- hosts: all
+  tasks: []
diff --git a/tests/fixtures/config/roles/git/org_project/roles/README b/tests/fixtures/config/roles/git/org_project/roles/README
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/tests/fixtures/config/roles/git/org_project/roles/README
diff --git a/tests/unit/test_connection.py b/tests/unit/test_connection.py
index 8d9d127..c9e2e91 100644
--- a/tests/unit/test_connection.py
+++ b/tests/unit/test_connection.py
@@ -69,7 +69,7 @@
         insp = sa.engine.reflection.Inspector(
             self.connections.connections['resultsdb'].engine)
 
-        self.assertEqual(9, len(insp.get_columns(buildset_table)))
+        self.assertEqual(10, len(insp.get_columns(buildset_table)))
         self.assertEqual(10, len(insp.get_columns(build_table)))
 
     def test_sql_results(self):
@@ -108,6 +108,7 @@
         self.assertEqual(1, buildset0['patchset'])
         self.assertEqual(1, buildset0['score'])
         self.assertEqual('Build succeeded.', buildset0['message'])
+        self.assertEqual('tenant-one', buildset0['tenant'])
 
         buildset0_builds = conn.execute(
             sa.sql.select([reporter.connection.zuul_build_table]).
diff --git a/tests/unit/test_github_driver.py b/tests/unit/test_github_driver.py
index f360866..0cfe3da 100644
--- a/tests/unit/test_github_driver.py
+++ b/tests/unit/test_github_driver.py
@@ -46,7 +46,7 @@
                          self.getJobFromHistory('project-test2').result)
 
         job = self.getJobFromHistory('project-test2')
-        zuulvars = job.parameters['vars']['zuul']
+        zuulvars = job.parameters['zuul']
         self.assertEqual(A.number, zuulvars['change'])
         self.assertEqual(A.head_sha, zuulvars['patchset'])
         self.assertEqual(1, len(A.comments))
diff --git a/tests/unit/test_model.py b/tests/unit/test_model.py
index 7fe101e..a52a2ee 100644
--- a/tests/unit/test_model.py
+++ b/tests/unit/test_model.py
@@ -54,7 +54,8 @@
                 encryption.deserialize_rsa_keypair(f.read())
         self.context = model.SourceContext(self.project, 'master',
                                            'test', True)
-        self.start_mark = yaml.Mark('name', 0, 0, 0, '', 0)
+        m = yaml.Mark('name', 0, 0, 0, '', 0)
+        self.start_mark = configloader.ZuulMark(m, m, '')
 
     @property
     def job(self):
@@ -95,9 +96,9 @@
     def test_job_inheritance(self):
         # This is standard job inheritance.
 
-        base_pre = model.PlaybookContext(self.context, 'base-pre')
-        base_run = model.PlaybookContext(self.context, 'base-run')
-        base_post = model.PlaybookContext(self.context, 'base-post')
+        base_pre = model.PlaybookContext(self.context, 'base-pre', [])
+        base_run = model.PlaybookContext(self.context, 'base-run', [])
+        base_post = model.PlaybookContext(self.context, 'base-post', [])
 
         base = model.Job('base')
         base.timeout = 30
@@ -121,9 +122,9 @@
     def test_job_variants(self):
         # This simulates freezing a job.
 
-        py27_pre = model.PlaybookContext(self.context, 'py27-pre')
-        py27_run = model.PlaybookContext(self.context, 'py27-run')
-        py27_post = model.PlaybookContext(self.context, 'py27-post')
+        py27_pre = model.PlaybookContext(self.context, 'py27-pre', [])
+        py27_run = model.PlaybookContext(self.context, 'py27-run', [])
+        py27_post = model.PlaybookContext(self.context, 'py27-post', [])
 
         py27 = model.Job('py27')
         py27.timeout = 30
diff --git a/tests/unit/test_scheduler.py b/tests/unit/test_scheduler.py
index 61bf9f8..ac2a779 100755
--- a/tests/unit/test_scheduler.py
+++ b/tests/unit/test_scheduler.py
@@ -2757,7 +2757,7 @@
 
         for build in self.history:
             self.assertEqual(results.get(build.uuid, ''),
-                             build.parameters['vars']['zuul'].get('tags'))
+                             build.parameters['zuul'].get('tags'))
 
     def test_timer(self):
         "Test that a periodic job is triggered"
diff --git a/tests/unit/test_v3.py b/tests/unit/test_v3.py
index 87eddc6..734c45c 100644
--- a/tests/unit/test_v3.py
+++ b/tests/unit/test_v3.py
@@ -199,6 +199,52 @@
         self.executor_server.release()
         self.waitUntilSettled()
 
+    def test_dynamic_dependent_pipeline(self):
+        # Test dynamically adding a project to a
+        # dependent pipeline for the first time
+        self.executor_server.hold_jobs_in_build = True
+
+        tenant = self.sched.abide.tenants.get('tenant-one')
+        gate_pipeline = tenant.layout.pipelines['gate']
+
+        in_repo_conf = textwrap.dedent(
+            """
+            - job:
+                name: project-test2
+
+            - project:
+                name: org/project
+                gate:
+                  jobs:
+                    - project-test2
+            """)
+
+        in_repo_playbook = textwrap.dedent(
+            """
+            - hosts: all
+              tasks: []
+            """)
+
+        file_dict = {'.zuul.yaml': in_repo_conf,
+                     'playbooks/project-test2.yaml': in_repo_playbook}
+        A = self.fake_gerrit.addFakeChange('org/project', 'master', 'A',
+                                           files=file_dict)
+        A.addApproval('approved', 1)
+        self.fake_gerrit.addEvent(A.addApproval('code-review', 2))
+        self.waitUntilSettled()
+
+        items = gate_pipeline.getAllItems()
+        self.assertEqual(items[0].change.number, '1')
+        self.assertEqual(items[0].change.patchset, '1')
+        self.assertTrue(items[0].live)
+
+        self.executor_server.hold_jobs_in_build = False
+        self.executor_server.release()
+        self.waitUntilSettled()
+
+        # Make sure the dynamic queue got cleaned up
+        self.assertEqual(gate_pipeline.queues, [])
+
     def test_in_repo_branch(self):
         in_repo_conf = textwrap.dedent(
             """
@@ -673,6 +719,25 @@
 class TestRoles(ZuulTestCase):
     tenant_config_file = 'config/roles/main.yaml'
 
+    def _assertRolePath(self, build, playbook, content):
+        path = os.path.join(self.test_root, build.uuid,
+                            'ansible', playbook, 'ansible.cfg')
+        roles_paths = []
+        with open(path) as f:
+            for line in f:
+                if line.startswith('roles_path'):
+                    roles_paths.append(line)
+        print(roles_paths)
+        if content:
+            self.assertEqual(len(roles_paths), 1,
+                             "Should have one roles_path line in %s" %
+                             (playbook,))
+            self.assertIn(content, roles_paths[0])
+        else:
+            self.assertEqual(len(roles_paths), 0,
+                             "Should have no roles_path line in %s" %
+                             (playbook,))
+
     def test_role(self):
         # This exercises a proposed change to a role being checked out
         # and used.
@@ -687,6 +752,75 @@
             dict(name='project-test', result='SUCCESS', changes='1,1 2,1'),
         ])
 
+    def test_role_inheritance(self):
+        self.executor_server.hold_jobs_in_build = True
+        conf = textwrap.dedent(
+            """
+            - job:
+                name: parent
+                roles:
+                  - zuul: bare-role
+                pre-run: playbooks/parent-pre
+                post-run: playbooks/parent-post
+
+            - job:
+                name: project-test
+                parent: parent
+                roles:
+                  - zuul: org/project
+
+            - project:
+                name: org/project
+                check:
+                  jobs:
+                    - project-test
+            """)
+
+        file_dict = {'.zuul.yaml': conf}
+        A = self.fake_gerrit.addFakeChange('org/project', 'master', 'A',
+                                           files=file_dict)
+        self.fake_gerrit.addEvent(A.getPatchsetCreatedEvent(1))
+        self.waitUntilSettled()
+
+        self.assertEqual(len(self.builds), 1)
+        build = self.getBuildByName('project-test')
+        self._assertRolePath(build, 'pre_playbook_0', 'role_0')
+        self._assertRolePath(build, 'playbook_0', 'role_0')
+        self._assertRolePath(build, 'playbook_0', 'role_1')
+        self._assertRolePath(build, 'post_playbook_0', 'role_0')
+
+        self.executor_server.hold_jobs_in_build = False
+        self.executor_server.release()
+        self.waitUntilSettled()
+
+        self.assertHistory([
+            dict(name='project-test', result='SUCCESS', changes='1,1'),
+        ])
+
+    def test_role_error(self):
+        conf = textwrap.dedent(
+            """
+            - job:
+                name: project-test
+                roles:
+                  - zuul: common-config
+
+            - project:
+                name: org/project
+                check:
+                  jobs:
+                    - project-test
+            """)
+
+        file_dict = {'.zuul.yaml': conf}
+        A = self.fake_gerrit.addFakeChange('org/project', 'master', 'A',
+                                           files=file_dict)
+        self.fake_gerrit.addEvent(A.getPatchsetCreatedEvent(1))
+        self.waitUntilSettled()
+        self.assertIn(
+            '- project-test project-test : ERROR Unable to find role',
+            A.messages[-1])
+
 
 class TestShadow(ZuulTestCase):
     tenant_config_file = 'config/shadow/main.yaml'
diff --git a/zuul/ansible/callback/zuul_json.py b/zuul/ansible/callback/zuul_json.py
new file mode 100644
index 0000000..017c27e
--- /dev/null
+++ b/zuul/ansible/callback/zuul_json.py
@@ -0,0 +1,138 @@
+# (c) 2016, Matt Martz <matt@sivel.net>
+# (c) 2017, Red Hat, Inc.
+#
+# Ansible is free software: you can redistribute it and/or modify
+# it under the terms of the GNU General Public License as published by
+# the Free Software Foundation, either version 3 of the License, or
+# (at your option) any later version.
+#
+# Ansible is distributed in the hope that it will be useful,
+# but WITHOUT ANY WARRANTY; without even the implied warranty of
+# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+# GNU General Public License for more details.
+#
+# You should have received a copy of the GNU General Public License
+# along with Ansible.  If not, see <http://www.gnu.org/licenses/>.
+
+# Copy of github.com/ansible/ansible/lib/ansible/plugins/callback/json.py
+# We need to run as a secondary callback not a stdout and we need to control
+# the output file location via a zuul environment variable similar to how we
+# do in zuul_stream.
+# Subclassing wreaks havoc on the module loader and namepsaces
+from __future__ import (absolute_import, division, print_function)
+__metaclass__ = type
+
+import json
+import os
+
+from ansible.plugins.callback import CallbackBase
+try:
+    # It's here in 2.4
+    from ansible.vars import strip_internal_keys
+except ImportError:
+    # It's here in 2.3
+    from ansible.vars.manager import strip_internal_keys
+
+
+class CallbackModule(CallbackBase):
+    CALLBACK_VERSION = 2.0
+    # aggregate means we can be loaded and not be the stdout plugin
+    CALLBACK_TYPE = 'aggregate'
+    CALLBACK_NAME = 'zuul_json'
+
+    def __init__(self, display=None):
+        super(CallbackModule, self).__init__(display)
+        self.results = []
+        self.output_path = os.path.splitext(
+            os.environ['ZUUL_JOB_OUTPUT_FILE'])[0] + '.json'
+        # For now, just read in the old file and write it all out again
+        # This may well not scale from a memory perspective- but let's see how
+        # it goes.
+        if os.path.exists(self.output_path):
+            self.results = json.load(open(self.output_path, 'r'))
+
+    def _get_playbook_name(self, work_dir):
+
+        playbook = self._playbook_name
+        if work_dir and playbook.startswith(work_dir):
+            playbook = playbook.replace(work_dir.rstrip('/') + '/', '')
+            # Lop off the first two path elements - ansible/pre_playbook_0
+            for prefix in ('pre', 'playbook', 'post'):
+                full_prefix = 'ansible/{prefix}_'.format(prefix=prefix)
+                if playbook.startswith(full_prefix):
+                    playbook = playbook.split(os.path.sep, 2)[2]
+        return playbook
+
+    def _new_play(self, play, phase, index, work_dir):
+        return {
+            'play': {
+                'name': play.name,
+                'id': str(play._uuid),
+                'phase': phase,
+                'index': index,
+                'playbook': self._get_playbook_name(work_dir),
+            },
+            'tasks': []
+        }
+
+    def _new_task(self, task):
+        return {
+            'task': {
+                'name': task.name,
+                'id': str(task._uuid)
+            },
+            'hosts': {}
+        }
+
+    def v2_playbook_on_start(self, playbook):
+        self._playbook_name = os.path.splitext(playbook._file_name)[0]
+
+    def v2_playbook_on_play_start(self, play):
+        # Get the hostvars from just one host - the vars we're looking for will
+        # be identical on all of them
+        hostvars = next(iter(play._variable_manager._hostvars.values()))
+        phase = hostvars.get('zuul_execution_phase')
+        index = hostvars.get('zuul_execution_phase_index')
+        # TODO(mordred) For now, protect this to make it not absurdly strange
+        # to run local tests with the callback plugin enabled. Remove once we
+        # have a "run playbook like zuul runs playbook" tool.
+        work_dir = None
+        if 'zuul' in hostvars and 'executor' in hostvars['zuul']:
+            # imply work_dir from src_root
+            work_dir = os.path.dirname(
+                hostvars['zuul']['executor']['src_root'])
+        self.results.append(self._new_play(play, phase, index, work_dir))
+
+    def v2_playbook_on_task_start(self, task, is_conditional):
+        self.results[-1]['tasks'].append(self._new_task(task))
+
+    def v2_runner_on_ok(self, result, **kwargs):
+        host = result._host
+        if result._result.get('_ansible_no_log', False):
+            self.results[-1]['tasks'][-1]['hosts'][host.name] = dict(
+                censored="the output has been hidden due to the fact that"
+                         " 'no_log: true' was specified for this result")
+        else:
+            clean_result = strip_internal_keys(result._result)
+            self.results[-1]['tasks'][-1]['hosts'][host.name] = clean_result
+
+    def v2_playbook_on_stats(self, stats):
+        """Display info about playbook statistics"""
+        hosts = sorted(stats.processed.keys())
+
+        summary = {}
+        for h in hosts:
+            s = stats.summarize(h)
+            summary[h] = s
+
+        output = {
+            'plays': self.results,
+            'stats': summary
+        }
+
+        json.dump(output, open(self.output_path, 'w'),
+                  indent=4, sort_keys=True, separators=(',', ': '))
+
+    v2_runner_on_failed = v2_runner_on_ok
+    v2_runner_on_unreachable = v2_runner_on_ok
+    v2_runner_on_skipped = v2_runner_on_ok
diff --git a/zuul/ansible/callback/zuul_stream.py b/zuul/ansible/callback/zuul_stream.py
index cc979f2..e9f969a 100644
--- a/zuul/ansible/callback/zuul_stream.py
+++ b/zuul/ansible/callback/zuul_stream.py
@@ -312,8 +312,7 @@
             else:
                 self._log_message(
                     result=result,
-                    status=status,
-                    result_dict=result_dict)
+                    status=status)
         elif 'results' in result_dict:
             for res in result_dict['results']:
                 self._log_message(
@@ -342,7 +341,6 @@
             self._log_message(
                 result=result,
                 msg="Item: {item}".format(item=result_dict['item']),
-                result_dict=result_dict,
                 status=status)
         else:
             self._log_message(
diff --git a/zuul/configloader.py b/zuul/configloader.py
index 254527a..f8e2d15 100644
--- a/zuul/configloader.py
+++ b/zuul/configloader.py
@@ -15,8 +15,8 @@
 import copy
 import os
 import logging
-import pprint
 import textwrap
+import io
 
 import voluptuous as vs
 
@@ -131,7 +131,7 @@
 
         {error}
 
-        The error appears in a {stanza} stanza with the content:
+        The error appears in the following {stanza} stanza:
 
         {content}
 
@@ -140,20 +140,42 @@
         m = m.format(intro=intro,
                      error=indent(str(e)),
                      stanza=stanza,
-                     content=indent(pprint.pformat(conf)),
+                     content=indent(start_mark.snippet.rstrip()),
                      start_mark=str(start_mark))
         raise ConfigurationSyntaxError(m)
 
 
+class ZuulMark(object):
+    # The yaml mark class differs between the C and python versions.
+    # The C version does not provide a snippet, and also appears to
+    # lose data under some circumstances.
+    def __init__(self, start_mark, end_mark, stream):
+        self.name = start_mark.name
+        self.index = start_mark.index
+        self.line = start_mark.line
+        self.column = start_mark.column
+        self.snippet = stream[start_mark.index:end_mark.index]
+
+    def __str__(self):
+        return '  in "{name}", line {line}, column {column}'.format(
+            name=self.name,
+            line=self.line + 1,
+            column=self.column + 1,
+        )
+
+
 class ZuulSafeLoader(yaml.SafeLoader):
     zuul_node_types = frozenset(('job', 'nodeset', 'secret', 'pipeline',
                                  'project', 'project-template',
                                  'semaphore'))
 
     def __init__(self, stream, context):
-        super(ZuulSafeLoader, self).__init__(stream)
+        wrapped_stream = io.StringIO(stream)
+        wrapped_stream.name = str(context)
+        super(ZuulSafeLoader, self).__init__(wrapped_stream)
         self.name = str(context)
         self.zuul_context = context
+        self.zuul_stream = stream
 
     def construct_mapping(self, node, deep=False):
         r = super(ZuulSafeLoader, self).construct_mapping(node, deep)
@@ -161,7 +183,8 @@
         if len(keys) == 1 and keys.intersection(self.zuul_node_types):
             d = list(r.values())[0]
             if isinstance(d, dict):
-                d['_start_mark'] = node.start_mark
+                d['_start_mark'] = ZuulMark(node.start_mark, node.end_mark,
+                                            self.zuul_stream)
                 d['_source_context'] = self.zuul_context
         return r
 
@@ -224,7 +247,7 @@
                    vs.Required('nodes'): to_list(node),
                    'groups': to_list(group),
                    '_source_context': model.SourceContext,
-                   '_start_mark': yaml.Mark,
+                   '_start_mark': ZuulMark,
                    }
 
         return vs.Schema(nodeset)
@@ -262,7 +285,7 @@
         secret = {vs.Required('name'): str,
                   vs.Required('data'): data,
                   '_source_context': model.SourceContext,
-                  '_start_mark': yaml.Mark,
+                  '_start_mark': ZuulMark,
                   }
 
         return vs.Schema(secret)
@@ -319,7 +342,7 @@
                'post-run': to_list(str),
                'run': str,
                '_source_context': model.SourceContext,
-               '_start_mark': yaml.Mark,
+               '_start_mark': ZuulMark,
                'roles': to_list(role),
                'required-projects': to_list(vs.Any(job_project, str)),
                'vars': dict,
@@ -400,21 +423,34 @@
             parent = layout.getJob(conf['parent'])
             job.inheritFrom(parent)
 
+        # Roles are part of the playbook context so we must establish
+        # them earlier than playbooks.
+        if 'roles' in conf:
+            roles = []
+            for role in conf.get('roles', []):
+                if 'zuul' in role:
+                    r = JobParser._makeZuulRole(tenant, job, role)
+                    if r:
+                        roles.append(r)
+            job.addRoles(roles)
+
         for pre_run_name in as_list(conf.get('pre-run')):
             pre_run = model.PlaybookContext(job.source_context,
-                                            pre_run_name)
+                                            pre_run_name, job.roles)
             job.pre_run = job.pre_run + (pre_run,)
         for post_run_name in as_list(conf.get('post-run')):
             post_run = model.PlaybookContext(job.source_context,
-                                             post_run_name)
+                                             post_run_name, job.roles)
             job.post_run = (post_run,) + job.post_run
         if 'run' in conf:
-            run = model.PlaybookContext(job.source_context, conf['run'])
+            run = model.PlaybookContext(job.source_context, conf['run'],
+                                        job.roles)
             job.run = (run,)
         else:
             if not project_pipeline:
                 run_name = os.path.join('playbooks', job.name)
-                run = model.PlaybookContext(job.source_context, run_name)
+                run = model.PlaybookContext(job.source_context, run_name,
+                                            job.roles)
                 job.implied_run = (run,) + job.implied_run
 
         for k in JobParser.simple_attributes:
@@ -460,15 +496,6 @@
 
         job.dependencies = frozenset(as_list(conf.get('dependencies')))
 
-        if 'roles' in conf:
-            roles = []
-            for role in conf.get('roles', []):
-                if 'zuul' in role:
-                    r = JobParser._makeZuulRole(tenant, job, role)
-                    if r:
-                        roles.append(r)
-            job.addRoles(roles)
-
         variables = conf.get('vars', None)
         if variables:
             job.updateVariables(variables)
@@ -539,7 +566,7 @@
                 'merge', 'merge-resolve',
                 'cherry-pick'),
             '_source_context': model.SourceContext,
-            '_start_mark': yaml.Mark,
+            '_start_mark': ZuulMark,
         }
 
         for p in layout.pipelines.values():
@@ -603,7 +630,7 @@
                                  'cherry-pick'),
             'default-branch': str,
             '_source_context': model.SourceContext,
-            '_start_mark': yaml.Mark,
+            '_start_mark': ZuulMark,
         }
 
         for p in layout.pipelines.values():
@@ -758,7 +785,7 @@
                     'window-decrease-type': window_type,
                     'window-decrease-factor': window_factor,
                     '_source_context': model.SourceContext,
-                    '_start_mark': yaml.Mark,
+                    '_start_mark': ZuulMark,
                     }
         pipeline['require'] = PipelineParser.getDriverSchema('require',
                                                              connections)
@@ -865,7 +892,7 @@
         semaphore = {vs.Required('name'): str,
                      'max': int,
                      '_source_context': model.SourceContext,
-                     '_start_mark': yaml.Mark,
+                     '_start_mark': ZuulMark,
                      }
 
         return vs.Schema(semaphore)
diff --git a/zuul/driver/bubblewrap/__init__.py b/zuul/driver/bubblewrap/__init__.py
index 5ec2448..3609a71 100644
--- a/zuul/driver/bubblewrap/__init__.py
+++ b/zuul/driver/bubblewrap/__init__.py
@@ -150,6 +150,7 @@
             '--ro-bind', '/bin', '/bin',
             '--ro-bind', '/sbin', '/sbin',
             '--ro-bind', '/etc/resolv.conf', '/etc/resolv.conf',
+            '--ro-bind', '/etc/hosts', '/etc/hosts',
             '--ro-bind', '{ssh_auth_sock}', '{ssh_auth_sock}',
             '--dir', '{work_dir}',
             '--bind', '{work_dir}', '{work_dir}',
@@ -166,6 +167,9 @@
 
         if os.path.isdir('/lib64'):
             bwrap_command.extend(['--ro-bind', '/lib64', '/lib64'])
+        if os.path.isfile('/etc/nsswitch.conf'):
+            bwrap_command.extend(['--ro-bind', '/etc/nsswitch.conf',
+                                  '/etc/nsswitch.conf'])
 
         return bwrap_command
 
diff --git a/zuul/driver/github/githubconnection.py b/zuul/driver/github/githubconnection.py
index 1a9e37b..a4a4c12 100644
--- a/zuul/driver/github/githubconnection.py
+++ b/zuul/driver/github/githubconnection.py
@@ -824,6 +824,9 @@
         # Create a repo object
         repository = github.repository(owner, proj)
 
+        if not repository:
+            return 'none'
+
         # Build up a URL
         url = repository._build_url('collaborators', login, 'permission',
                                     base_url=repository._api)
diff --git a/zuul/driver/sql/alembic_reporter/versions/f86c9871ee67_add_tenant_column.py b/zuul/driver/sql/alembic_reporter/versions/f86c9871ee67_add_tenant_column.py
new file mode 100644
index 0000000..7728bd4
--- /dev/null
+++ b/zuul/driver/sql/alembic_reporter/versions/f86c9871ee67_add_tenant_column.py
@@ -0,0 +1,38 @@
+# Copyright 2017 Red Hat, 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.
+
+"""Add tenant column
+
+Revision ID: f86c9871ee67
+Revises: 20126015a87d
+Create Date: 2017-07-17 05:47:48.189767
+
+"""
+
+# revision identifiers, used by Alembic.
+revision = 'f86c9871ee67'
+down_revision = '20126015a87d'
+branch_labels = None
+depends_on = None
+
+from alembic import op
+import sqlalchemy as sa
+
+
+def upgrade():
+    op.add_column('zuul_buildset', sa.Column('tenant', sa.String(255)))
+
+
+def downgrade():
+    op.drop_column('zuul_buildset', 'tenant')
diff --git a/zuul/driver/sql/sqlconnection.py b/zuul/driver/sql/sqlconnection.py
index e478d33..0e3f0dd 100644
--- a/zuul/driver/sql/sqlconnection.py
+++ b/zuul/driver/sql/sqlconnection.py
@@ -85,6 +85,7 @@
             sa.Column('ref', sa.String(255)),
             sa.Column('score', sa.Integer, nullable=True),
             sa.Column('message', sa.TEXT()),
+            sa.Column('tenant', sa.String(255)),
         )
 
         zuul_build_table = sa.Table(
diff --git a/zuul/driver/sql/sqlreporter.py b/zuul/driver/sql/sqlreporter.py
index 5f93ce8..214b667 100644
--- a/zuul/driver/sql/sqlreporter.py
+++ b/zuul/driver/sql/sqlreporter.py
@@ -52,6 +52,7 @@
                 score=self.result_score,
                 message=self._formatItemReport(
                     item, with_jobs=False),
+                tenant=item.pipeline.layout.tenant.name,
             )
             buildset_ins_result = conn.execute(buildset_ins)
             build_inserts = []
diff --git a/zuul/executor/client.py b/zuul/executor/client.py
index c36d569..f764778 100644
--- a/zuul/executor/client.py
+++ b/zuul/executor/client.py
@@ -237,7 +237,6 @@
             params['playbooks'] = [x.toDict() for x in job.run]
             params['pre_playbooks'] = [x.toDict() for x in job.pre_run]
             params['post_playbooks'] = [x.toDict() for x in job.post_run]
-            params['roles'] = [x.toDict() for x in job.roles]
 
         nodeset = item.current_build_set.getJobNodeSet(job.name)
         nodes = []
@@ -254,10 +253,12 @@
         params['nodes'] = nodes
         params['groups'] = [group.toDict() for group in nodeset.getGroups()]
         params['vars'] = copy.deepcopy(job.variables)
+        params['secrets'] = {}
         if job.auth:
             for secret in job.auth.secrets:
-                params['vars'][secret.name] = copy.deepcopy(secret.secret_data)
-        params['vars']['zuul'] = zuul_params
+                secret_data = copy.deepcopy(secret.secret_data)
+                params['secrets'][secret.name] = secret_data
+        params['zuul'] = zuul_params
         projects = set()
 
         def make_project_dict(project, override_branch=None):
@@ -377,6 +378,7 @@
             build.node_name = data.get('node_name')
             if result is None:
                 result = data.get('result')
+                build.error_detail = data.get('error_detail')
             if result is None:
                 if (build.build_set.getTries(build.job.name) >=
                     build.job.attempts):
diff --git a/zuul/executor/server.py b/zuul/executor/server.py
index cdd082e..f291dce 100644
--- a/zuul/executor/server.py
+++ b/zuul/executor/server.py
@@ -41,6 +41,16 @@
 DEFAULT_FINGER_PORT = 79
 
 
+class ExecutorError(Exception):
+    """A non-transient run-time executor error
+
+    This class represents error conditions detected by the executor
+    when preparing to run a job which we know are consistently fatal.
+    Zuul should not reschedule the build in these cases.
+    """
+    pass
+
+
 class Watchdog(object):
     def __init__(self, timeout, function, args):
         self.timeout = timeout
@@ -72,13 +82,6 @@
         self._running = False
 
 
-class JobDirPlaybook(object):
-    def __init__(self, root):
-        self.root = root
-        self.trusted = None
-        self.path = None
-
-
 class SshAgent(object):
     log = logging.getLogger("zuul.ExecutorServer")
 
@@ -122,8 +125,8 @@
             subprocess.check_output(['ssh-add', key_path], env=env,
                                     stderr=subprocess.PIPE)
         except subprocess.CalledProcessError as e:
-            self.log.error('ssh-add failed. stdout: %s, stderr: %s',
-                           e.output, e.stderr)
+            self.log.exception('ssh-add failed. stdout: %s, stderr: %s',
+                               e.output, e.stderr)
             raise
         self.log.info('Added SSH Key {}'.format(key_path))
 
@@ -151,6 +154,24 @@
         return result
 
 
+class JobDirPlaybook(object):
+    def __init__(self, root):
+        self.root = root
+        self.trusted = None
+        self.path = None
+        self.roles = []
+        self.roles_path = []
+        self.ansible_config = os.path.join(self.root, 'ansible.cfg')
+        self.project_link = os.path.join(self.root, 'project')
+
+    def addRole(self):
+        count = len(self.roles)
+        root = os.path.join(self.root, 'role_%i' % (count,))
+        os.makedirs(root)
+        self.roles.append(root)
+        return root
+
+
 class JobDir(object):
     def __init__(self, root, keep, build_uuid):
         '''
@@ -163,11 +184,23 @@
         '''
         # root
         #   ansible
-        #     trusted.cfg
-        #     untrusted.cfg
+        #     inventory.yaml
+        #   playbook_0
+        #     project -> ../trusted/project_0/...
+        #     role_0 -> ../trusted/project_0/...
+        #   trusted
+        #     project_0
+        #       <git.example.com>
+        #         <project>
         #   work
+        #     .ssh
+        #       known_hosts
         #     src
+        #       <git.example.com>
+        #         <project>
         #     logs
+        #       job-output.txt
+        #     results.json
         self.keep = keep
         if root:
             tmpdir = root
@@ -175,16 +208,16 @@
             tmpdir = tempfile.gettempdir()
         self.root = os.path.join(tmpdir, build_uuid)
         os.mkdir(self.root, 0o700)
-        # Work
         self.work_root = os.path.join(self.root, 'work')
         os.makedirs(self.work_root)
         self.src_root = os.path.join(self.work_root, 'src')
         os.makedirs(self.src_root)
         self.log_root = os.path.join(self.work_root, 'logs')
         os.makedirs(self.log_root)
-        # Ansible
         self.ansible_root = os.path.join(self.root, 'ansible')
         os.makedirs(self.ansible_root)
+        self.trusted_root = os.path.join(self.root, 'trusted')
+        os.makedirs(self.trusted_root)
         ssh_dir = os.path.join(self.work_root, '.ssh')
         os.mkdir(ssh_dir, 0o700)
         self.result_data_file = os.path.join(self.work_root, 'results.json')
@@ -192,17 +225,29 @@
             pass
         self.known_hosts = os.path.join(ssh_dir, 'known_hosts')
         self.inventory = os.path.join(self.ansible_root, 'inventory.yaml')
+        self.secrets = os.path.join(self.ansible_root, 'secrets.yaml')
+        self.has_secrets = False
         self.playbooks = []  # The list of candidate playbooks
         self.playbook = None  # A pointer to the candidate we have chosen
         self.pre_playbooks = []
         self.post_playbooks = []
-        self.roles = []
-        self.trusted_roles_path = []
-        self.untrusted_roles_path = []
-        self.untrusted_config = os.path.join(
-            self.ansible_root, 'untrusted.cfg')
-        self.trusted_config = os.path.join(self.ansible_root, 'trusted.cfg')
         self.job_output_file = os.path.join(self.log_root, 'job-output.txt')
+        self.trusted_projects = []
+        self.trusted_project_index = {}
+
+    def addTrustedProject(self, canonical_name, branch):
+        # Trusted projects are placed in their own directories so that
+        # we can support using different branches of the same project
+        # in different playbooks.
+        count = len(self.trusted_projects)
+        root = os.path.join(self.trusted_root, 'project_%i' % (count,))
+        os.makedirs(root)
+        self.trusted_projects.append(root)
+        self.trusted_project_index[(canonical_name, branch)] = root
+        return root
+
+    def getTrustedProject(self, canonical_name, branch):
+        return self.trusted_project_index.get((canonical_name, branch))
 
     def addPrePlaybook(self):
         count = len(self.pre_playbooks)
@@ -228,17 +273,6 @@
         self.playbooks.append(playbook)
         return playbook
 
-    def addRole(self):
-        count = len(self.roles)
-        root = os.path.join(self.ansible_root, 'role_%i' % (count,))
-        os.makedirs(root)
-        trusted = os.path.join(root, 'trusted')
-        os.makedirs(trusted)
-        untrusted = os.path.join(root, 'untrusted')
-        os.makedirs(untrusted)
-        self.roles.append(root)
-        return root
-
     def cleanup(self):
         if not self.keep:
             shutil.rmtree(self.root)
@@ -722,6 +756,11 @@
                                  self.executor_server.keep_jobdir,
                                  str(self.job.unique))
             self._execute()
+        except ExecutorError as e:
+            result_data = json.dumps(dict(result='ERROR',
+                                          error_detail=e.args[0]))
+            self.log.debug("Sending result: %s" % (result_data,))
+            self.job.sendWorkComplete(result_data)
         except Exception:
             self.log.exception("Exception while executing job")
             self.job.sendWorkException(traceback.format_exc())
@@ -745,7 +784,7 @@
     def _execute(self):
         args = json.loads(self.job.arguments)
         self.log.debug("Beginning job %s for ref %s" %
-                       (self.job.name, args['vars']['zuul']['ref']))
+                       (self.job.name, args['zuul']['ref']))
         self.log.debug("Args: %s" % (self.job.arguments,))
         self.log.debug("Job root: %s" % (self.jobdir.root,))
         tasks = []
@@ -760,8 +799,13 @@
             projects.add((project['connection'], project['name']))
 
         # ...as well as all playbook and role projects.
-        repos = (args['pre_playbooks'] + args['playbooks'] +
-                 args['post_playbooks'] + args['roles'])
+        repos = []
+        playbooks = (args['pre_playbooks'] + args['playbooks'] +
+                     args['post_playbooks'])
+        for playbook in playbooks:
+            repos.append(playbook)
+            repos += playbook['roles']
+
         for repo in repos:
             self.log.debug("Job %s: updating playbook or role %s" %
                            (self.job.unique, repo))
@@ -806,12 +850,9 @@
         for repo in repos.values():
             repo.deleteRemote('origin')
 
-        # is the playbook in a repo that we have already prepared?
-        trusted, untrusted = self.preparePlaybookRepos(args)
+        # This prepares each playbook and the roles needed for each.
+        self.preparePlaybooks(args)
 
-        self.prepareRoles(args, trusted, untrusted)
-
-        # TODOv3: Ansible the ansible thing here.
         self.prepareAnsibleFiles(args)
 
         data = {
@@ -889,8 +930,9 @@
                           project_name, project_default_branch)
             repo.checkoutLocalBranch(project_default_branch)
         else:
-            raise Exception("Project %s does not have the default branch %s" %
-                            (project_name, project_default_branch))
+            raise ExecutorError("Project %s does not have the "
+                                "default branch %s" %
+                                (project_name, project_default_branch))
 
     def runPlaybooks(self, args):
         result = None
@@ -981,9 +1023,9 @@
         '''
         for entry in os.listdir(path):
             if os.path.isdir(entry) and entry.endswith('_plugins'):
-                raise Exception(
-                    "Ansible plugin dir %s found adjacent to playbook %s in"
-                    " non-trusted repo." % (entry, path))
+                raise ExecutorError(
+                    "Ansible plugin dir %s found adjacent to playbook %s in "
+                    "non-trusted repo." % (entry, path))
 
     def findPlaybook(self, path, required=False, trusted=False):
         for ext in ['.yaml', '.yml']:
@@ -994,45 +1036,32 @@
                     self._blockPluginDirs(playbook_dir)
                 return fn
         if required:
-            raise Exception("Unable to find playbook %s" % path)
+            raise ExecutorError("Unable to find playbook %s" % path)
         return None
 
-    def preparePlaybookRepos(self, args):
-        trusted = untrusted = False
+    def preparePlaybooks(self, args):
         for playbook in args['pre_playbooks']:
             jobdir_playbook = self.jobdir.addPrePlaybook()
-            self.preparePlaybookRepo(jobdir_playbook, playbook,
-                                     args, required=True)
-            if playbook['trusted']:
-                trusted = True
-            else:
-                untrusted = True
+            self.preparePlaybook(jobdir_playbook, playbook,
+                                 args, required=True)
 
         for playbook in args['playbooks']:
             jobdir_playbook = self.jobdir.addPlaybook()
-            self.preparePlaybookRepo(jobdir_playbook, playbook,
-                                     args, required=False)
-            if playbook['trusted']:
-                trusted = True
-            else:
-                untrusted = True
+            self.preparePlaybook(jobdir_playbook, playbook,
+                                 args, required=False)
             if jobdir_playbook.path is not None:
                 self.jobdir.playbook = jobdir_playbook
                 break
+
         if self.jobdir.playbook is None:
-            raise Exception("No valid playbook found")
+            raise ExecutorError("No valid playbook found")
 
         for playbook in args['post_playbooks']:
             jobdir_playbook = self.jobdir.addPostPlaybook()
-            self.preparePlaybookRepo(jobdir_playbook, playbook,
-                                     args, required=True)
-            if playbook['trusted']:
-                trusted = True
-            else:
-                untrusted = True
-        return (trusted, untrusted)
+            self.preparePlaybook(jobdir_playbook, playbook,
+                                 args, required=True)
 
-    def preparePlaybookRepo(self, jobdir_playbook, playbook, args, required):
+    def preparePlaybook(self, jobdir_playbook, playbook, args, required):
         self.log.debug("Prepare playbook repo for %s" % (playbook,))
         # Check out the playbook repo if needed and set the path to
         # the playbook that should be run.
@@ -1040,6 +1069,7 @@
         source = self.executor_server.connections.getSource(
             playbook['connection'])
         project = source.getProject(playbook['project'])
+        path = None
         if not playbook['trusted']:
             # This is a project repo, so it is safe to use the already
             # checked out version (from speculative merging) of the
@@ -1052,34 +1082,48 @@
                                         project.canonical_hostname,
                                         project.name,
                                         playbook['path'])
-                    jobdir_playbook.path = self.findPlaybook(
-                        path,
-                        required=required,
-                        trusted=playbook['trusted'])
-                    return
-        # The playbook repo is either a config repo, or it isn't in
-        # the stack of changes we are testing, so check out the branch
-        # tip into a dedicated space.
+                    break
+        if not path:
+            # The playbook repo is either a config repo, or it isn't in
+            # the stack of changes we are testing, so check out the branch
+            # tip into a dedicated space.
+            path = self.checkoutTrustedProject(project, playbook['branch'])
+            path = os.path.join(path, playbook['path'])
 
-        merger = self.executor_server._getMerger(jobdir_playbook.root,
-                                                 self.log)
-        merger.checkoutBranch(playbook['connection'], project.name,
-                              playbook['branch'])
-
-        path = os.path.join(jobdir_playbook.root,
-                            project.canonical_hostname,
-                            project.name,
-                            playbook['path'])
         jobdir_playbook.path = self.findPlaybook(
             path,
             required=required,
             trusted=playbook['trusted'])
 
-    def prepareRoles(self, args, trusted, untrusted):
-        for role in args['roles']:
-            if role['type'] == 'zuul':
-                root = self.jobdir.addRole()
-                self.prepareZuulRole(args, role, root, trusted, untrusted)
+        # If this playbook doesn't exist, don't bother preparing
+        # roles.
+        if not jobdir_playbook.path:
+            return
+
+        for role in playbook['roles']:
+            self.prepareRole(jobdir_playbook, role, args)
+
+        self.writeAnsibleConfig(jobdir_playbook)
+
+    def checkoutTrustedProject(self, project, branch):
+        root = self.jobdir.getTrustedProject(project.canonical_name,
+                                             branch)
+        if not root:
+            root = self.jobdir.addTrustedProject(project.canonical_name,
+                                                 branch)
+            merger = self.executor_server._getMerger(root, self.log)
+            merger.checkoutBranch(project.connection_name, project.name,
+                                  branch)
+
+        path = os.path.join(root,
+                            project.canonical_hostname,
+                            project.name)
+        return path
+
+    def prepareRole(self, jobdir_playbook, role, args):
+        if role['type'] == 'zuul':
+            root = jobdir_playbook.addRole()
+            self.prepareZuulRole(jobdir_playbook, role, args, root)
 
     def findRole(self, path, trusted=False):
         d = os.path.join(path, 'tasks')
@@ -1094,107 +1138,64 @@
             # This repo has a collection of roles
             if not trusted:
                 for entry in os.listdir(d):
-                    self._blockPluginDirs(os.path.join(d, entry))
+                    if os.path.isdir(os.path.join(d, entry)):
+                        self._blockPluginDirs(os.path.join(d, entry))
             return d
         # It is neither a bare role, nor a collection of roles
-        raise Exception("Unable to find role in %s" % (path,))
+        raise ExecutorError("Unable to find role in %s" % (path,))
 
-    def prepareZuulRole(self, args, role, root, trusted, untrusted):
+    def prepareZuulRole(self, jobdir_playbook, role, args, root):
         self.log.debug("Prepare zuul role for %s" % (role,))
         # Check out the role repo if needed
         source = self.executor_server.connections.getSource(
             role['connection'])
         project = source.getProject(role['project'])
-        untrusted_role_repo = None
-        trusted_role_repo = None
-        trusted_root = os.path.join(root, 'trusted')
-        untrusted_root = os.path.join(root, 'untrusted')
         name = role['target_name']
+        path = None
 
-        if untrusted:
-            # There is at least one untrusted playbook.  For that
-            # case, use the already checked out version (from
-            # speculative merging) of the role.
+        if not jobdir_playbook.trusted:
+            # This playbook is untrested.  Use the already checked out
+            # version (from speculative merging) of the role if it
+            # exists.
 
             for i in args['items']:
                 if (i['connection'] == role['connection'] and
                     i['project'] == role['project']):
-                    # We already have this repo prepared;
-                    # copy it into location.
-
+                    # We already have this repo prepared; use it.
                     path = os.path.join(self.jobdir.src_root,
                                         project.canonical_hostname,
                                         project.name)
-                    # The name of the symlink is the requested name of
-                    # the role (which may be the repo name or may be
-                    # something else; this can come into play if this
-                    # is a bare role).
-                    link = os.path.join(untrusted_root, name)
-                    link = os.path.realpath(link)
-                    if not link.startswith(os.path.realpath(untrusted_root)):
-                        raise Exception("Invalid role name %s", name)
-                    os.symlink(path, link)
-                    untrusted_role_repo = link
                     break
 
-        if trusted or not untrusted_role_repo:
-            # There is at least one trusted playbook which will need a
-            # trusted checkout of the role, or the role did not appear
+        if not path:
+            # This is a trusted playbook or the role did not appear
             # in the dependency chain for the change (in which case,
             # there is no existing untrusted checkout of it).  Check
             # out the branch tip into a dedicated space.
-            merger = self.executor_server._getMerger(trusted_root,
-                                                     self.log)
-            merger.checkoutBranch(role['connection'], project.name,
-                                  'master')
-            orig_repo_path = os.path.join(trusted_root,
-                                          project.canonical_hostname,
-                                          project.name)
-            if name != project.name:
-                # The requested name of the role is not the same as
-                # the project name, so rename the git repo as the
-                # requested name.  It is the only item in this
-                # directory, so we don't need to worry about
-                # collisions.
-                target = os.path.join(trusted_root,
-                                      project.canonical_hostname,
-                                      name)
-                target = os.path.realpath(target)
-                if not target.startswith(os.path.realpath(trusted_root)):
-                    raise Exception("Invalid role name %s", name)
-                os.rename(orig_repo_path, target)
-                trusted_role_repo = target
-            else:
-                trusted_role_repo = orig_repo_path
+            path = self.checkoutTrustedProject(project, 'master')
 
-            if not untrusted_role_repo:
-                # In the case that there was no untrusted checkout,
-                # use the trusted checkout.
-                untrusted_role_repo = trusted_role_repo
-                untrusted_root = trusted_root
+        # The name of the symlink is the requested name of the role
+        # (which may be the repo name or may be something else; this
+        # can come into play if this is a bare role).
+        link = os.path.join(root, name)
+        link = os.path.realpath(link)
+        if not link.startswith(os.path.realpath(root)):
+            raise ExecutorError("Invalid role name %s", name)
+        os.symlink(path, link)
 
-        if untrusted:
-            untrusted_role_path = self.findRole(untrusted_role_repo,
-                                                trusted=False)
-            if untrusted_role_path is None:
-                # In the case of a bare role, add the containing directory
-                untrusted_role_path = os.path.join(untrusted_root,
-                                                   project.canonical_hostname)
-            self.jobdir.untrusted_roles_path.append(untrusted_role_path)
-
-        if trusted:
-            trusted_role_path = self.findRole(trusted_role_repo,
-                                              trusted=True)
-            if trusted_role_path is None:
-                # In the case of a bare role, add the containing directory
-                trusted_role_path = os.path.join(trusted_root,
-                                                 project.canonical_hostname)
-            self.jobdir.trusted_roles_path.append(trusted_role_path)
+        role_path = self.findRole(link, trusted=jobdir_playbook.trusted)
+        if role_path is None:
+            # In the case of a bare role, add the containing directory
+            role_path = root
+        jobdir_playbook.roles_path.append(role_path)
 
     def prepareAnsibleFiles(self, args):
-        all_vars = dict(args['vars'])
+        all_vars = args['vars'].copy()
         # TODO(mordred) Hack to work around running things with python3
         all_vars['ansible_python_interpreter'] = '/usr/bin/python2'
+        if 'zuul' in all_vars:
+            raise Exception("Defining vars named 'zuul' is not allowed")
+        all_vars['zuul'] = args['zuul'].copy()
         all_vars['zuul']['executor'] = dict(
             hostname=self.executor_server.hostname,
             src_root=self.jobdir.src_root,
@@ -1213,11 +1214,19 @@
                 for key in node['host_keys']:
                     known_hosts.write('%s\n' % key)
 
-        self.writeAnsibleConfig(self.jobdir.untrusted_config)
-        self.writeAnsibleConfig(self.jobdir.trusted_config, trusted=True)
+        secrets = args['secrets'].copy()
+        if secrets:
+            if 'zuul' in secrets:
+                raise Exception("Defining secrets named 'zuul' is not allowed")
+            with open(self.jobdir.secrets, 'w') as secrets_yaml:
+                secrets_yaml.write(
+                    yaml.safe_dump(secrets, default_flow_style=False))
+            self.jobdir.has_secrets = True
 
-    def writeAnsibleConfig(self, config_path, trusted=False):
-        with open(config_path, 'w') as config:
+    def writeAnsibleConfig(self, jobdir_playbook):
+        trusted = jobdir_playbook.trusted
+
+        with open(jobdir_playbook.ansible_config, 'w') as config:
             config.write('[defaults]\n')
             config.write('hostfile = %s\n' % self.jobdir.inventory)
             config.write('local_tmp = %s/.ansible/local_tmp\n' %
@@ -1232,6 +1241,7 @@
             config.write('callback_plugins = %s\n'
                          % self.executor_server.callback_dir)
             config.write('stdout_callback = zuul_stream\n')
+            config.write('callback_whitelist = zuul_json\n')
             # bump the timeout because busy nodes may take more than
             # 10s to respond
             config.write('timeout = 30\n')
@@ -1240,12 +1250,10 @@
                              % self.executor_server.action_dir)
                 config.write('lookup_plugins = %s\n'
                              % self.executor_server.lookup_dir)
-                roles_path = self.jobdir.untrusted_roles_path
-            else:
-                roles_path = self.jobdir.trusted_roles_path
 
-            if roles_path:
-                config.write('roles_path = %s\n' % ':'.join(roles_path))
+            if jobdir_playbook.roles_path:
+                config.write('roles_path = %s\n' % ':'.join(
+                    jobdir_playbook.roles_path))
 
             # On trusted jobs, we want to prevent the printing of args,
             # since trusted jobs might have access to secrets that they may
@@ -1286,7 +1294,7 @@
             except Exception:
                 self.log.exception("Exception while killing ansible process:")
 
-    def runAnsible(self, cmd, timeout, trusted=False):
+    def runAnsible(self, cmd, timeout, config_file, trusted):
         env_copy = os.environ.copy()
         env_copy.update(self.ssh_agent.env)
         env_copy['LOGNAME'] = 'zuul'
@@ -1301,10 +1309,8 @@
         env_copy['PYTHONPATH'] = os.path.pathsep.join(pythonpath)
 
         if trusted:
-            config_file = self.jobdir.trusted_config
             opt_prefix = 'trusted'
         else:
-            config_file = self.jobdir.untrusted_config
             opt_prefix = 'untrusted'
         ro_dirs = get_default(self.executor_server.config, 'executor',
                               '%s_ro_dirs' % opt_prefix)
@@ -1378,6 +1384,8 @@
             # TODO(mordred) If/when we rework use of logger in ansible-playbook
             # we'll want to change how this works to use that as well. For now,
             # this is what we need to do.
+            # TODO(mordred) We probably want to put this into the json output
+            # as well.
             with open(self.jobdir.job_output_file, 'a') as job_output:
                 job_output.write("{now} | ANSIBLE PARSE ERROR\n".format(
                     now=datetime.datetime.now()))
@@ -1398,6 +1406,8 @@
             verbose = '-v'
 
         cmd = ['ansible-playbook', verbose, playbook.path]
+        if self.jobdir.has_secrets:
+            cmd.extend(['-e', '@' + self.jobdir.secrets])
 
         if success is not None:
             cmd.extend(['-e', 'success=%s' % str(bool(success))])
@@ -1409,7 +1419,9 @@
             cmd.extend(['-e', 'zuul_execution_phase_index=%s' % index])
 
         result, code = self.runAnsible(
-            cmd=cmd, timeout=timeout, trusted=playbook.trusted)
+            cmd=cmd, timeout=timeout,
+            config_file=playbook.ansible_config,
+            trusted=playbook.trusted)
         self.log.debug("Ansible complete, result %s code %s" % (
             self.RESULT_MAP[result], code))
         return result, code
diff --git a/zuul/manager/dependent.py b/zuul/manager/dependent.py
index ada3491..411894e 100644
--- a/zuul/manager/dependent.py
+++ b/zuul/manager/dependent.py
@@ -14,6 +14,7 @@
 
 from zuul import model
 from zuul.manager import PipelineManager, StaticChangeQueueContextManager
+from zuul.manager import DynamicChangeQueueContextManager
 
 
 class DependentPipelineManager(PipelineManager):
@@ -75,8 +76,17 @@
     def getChangeQueue(self, change, existing=None):
         if existing:
             return StaticChangeQueueContextManager(existing)
-        return StaticChangeQueueContextManager(
-            self.pipeline.getQueue(change.project))
+        queue = self.pipeline.getQueue(change.project)
+        if queue:
+            return StaticChangeQueueContextManager(queue)
+        else:
+            # There is no existing queue for this change. Create a
+            # dynamic one for this one change's use
+            change_queue = model.ChangeQueue(self.pipeline, dynamic=True)
+            change_queue.addProject(change.project)
+            self.pipeline.addQueue(change_queue)
+            self.log.debug("Dynamically created queue %s", change_queue)
+            return DynamicChangeQueueContextManager(change_queue)
 
     def isChangeReadyToBeEnqueued(self, change):
         source = change.project.source
@@ -201,3 +211,11 @@
         if failing_items:
             return failing_items
         return None
+
+    def dequeueItem(self, item):
+        super(DependentPipelineManager, self).dequeueItem(item)
+        # If this was a dynamic queue from a speculative change,
+        # remove the queue (if empty)
+        if item.queue.dynamic:
+            if not item.queue.queue:
+                self.pipeline.removeQueue(item.queue)
diff --git a/zuul/model.py b/zuul/model.py
index 80ef285..1df70db 100644
--- a/zuul/model.py
+++ b/zuul/model.py
@@ -209,11 +209,14 @@
     be processed. If a Change succeeds, the Window is increased by
     `window_increase_factor`. If a Change fails, the Window is decreased by
     `window_decrease_factor`.
+
+    A ChangeQueue may be a dynamically created queue, which may be removed
+    from a DependentPipelineManager once empty.
     """
     def __init__(self, pipeline, window=0, window_floor=1,
                  window_increase_type='linear', window_increase_factor=1,
                  window_decrease_type='exponential', window_decrease_factor=2,
-                 name=None):
+                 name=None, dynamic=False):
         self.pipeline = pipeline
         if name:
             self.name = name
@@ -228,6 +231,7 @@
         self.window_increase_factor = window_increase_factor
         self.window_decrease_type = window_decrease_type
         self.window_decrease_factor = window_decrease_factor
+        self.dynamic = dynamic
 
     def __repr__(self):
         return '<ChangeQueue %s: %s>' % (self.pipeline.name, self.name)
@@ -633,11 +637,17 @@
 
     Jobs refer to objects of this class for their main, pre, and post
     playbooks so that we can keep track of which repos and security
-    contexts are needed in order to run them."""
+    contexts are needed in order to run them.
 
-    def __init__(self, source_context, path):
+    We also keep a list of roles so that playbooks only run with the
+    roles which were defined at the point the playbook was defined.
+
+    """
+
+    def __init__(self, source_context, path, roles):
         self.source_context = source_context
         self.path = path
+        self.roles = roles
 
     def __repr__(self):
         return '<PlaybookContext %s %s>' % (self.source_context,
@@ -650,7 +660,8 @@
         if not isinstance(other, PlaybookContext):
             return False
         return (self.source_context == other.source_context and
-                self.path == other.path)
+                self.path == other.path and
+                self.roles == other.roles)
 
     def toDict(self):
         # Render to a dict to use in passing json to the executor
@@ -659,6 +670,7 @@
             project=self.source_context.project.name,
             branch=self.source_context.branch,
             trusted=self.source_context.trusted,
+            roles=[r.toDict() for r in self.roles],
             path=self.path)
 
 
@@ -704,7 +716,7 @@
         if not isinstance(other, ZuulRole):
             return False
         return (super(ZuulRole, self).__eq__(other) and
-                self.connection_name == other.connection_name,
+                self.connection_name == other.connection_name and
                 self.project_name == other.project_name)
 
     def toDict(self):
@@ -856,9 +868,9 @@
 
     def addRoles(self, roles):
         newroles = list(self.roles)
-        for role in roles:
+        for role in reversed(roles):
             if role not in newroles:
-                newroles.append(role)
+                newroles.insert(0, role)
         self.roles = tuple(newroles)
 
     def updateVariables(self, other_vars):
@@ -1086,6 +1098,7 @@
         self.url = None
         self.result = None
         self.result_data = {}
+        self.error_detail = None
         self.build_set = None
         self.execute_time = time.time()
         self.start_time = None
@@ -1106,6 +1119,7 @@
     def getSafeAttributes(self):
         return Attributes(uuid=self.uuid,
                           result=self.result,
+                          error_detail=self.error_detail,
                           result_data=self.result_data)
 
 
diff --git a/zuul/reporter/__init__.py b/zuul/reporter/__init__.py
index 95b9208..49181a7 100644
--- a/zuul/reporter/__init__.py
+++ b/zuul/reporter/__init__.py
@@ -138,7 +138,11 @@
                     elapsed = ' in %ds' % (s)
             else:
                 elapsed = ''
+            if build.error_detail:
+                error = ' ' + build.error_detail
+            else:
+                error = ''
             name = job.name + ' '
-            ret += '- %s%s : %s%s%s\n' % (name, url, result, elapsed,
-                                          voting)
+            ret += '- %s%s : %s%s%s%s\n' % (name, url, result, error,
+                                            elapsed, voting)
         return ret