Merge "executor: run trusted playbook in a bubblewrap" into feature/zuulv3
diff --git a/README.rst b/README.rst
index c55f7b3..16e7385 100644
--- a/README.rst
+++ b/README.rst
@@ -134,6 +134,16 @@
   is too cryptic.  In your own work, feel free to leave TODOv3 notes
   if a change would otherwise become too large or unweildy.
 
+Python Version Support
+----------------------
+
+Zuul v3 requires Python 3. It does not support Python 2.
+
+As Ansible is used for the execution of jobs, it's important to note that
+while Ansible does support Python 3, not all of Ansible's modules do. Zuul
+currently sets ``ansible_python_interpreter`` to python2 so that remote
+content will be executed with Python2.
+
 Roadmap
 -------
 
diff --git a/bindep.txt b/bindep.txt
index 5db144b..8dffd0f 100644
--- a/bindep.txt
+++ b/bindep.txt
@@ -15,3 +15,4 @@
 python-dev [platform:dpkg]
 python-devel [platform:rpm]
 bubblewrap [platform:rpm]
+redhat-rpm-config [platform:rpm]
diff --git a/requirements.txt b/requirements.txt
index 746bbcb..81f930e 100644
--- a/requirements.txt
+++ b/requirements.txt
@@ -8,7 +8,6 @@
 WebOb>=1.2.3
 paramiko>=1.8.0,<2.0.0
 GitPython>=0.3.3,<2.1.2
-ordereddict
 python-daemon>=2.0.4,<2.1.0
 extras
 statsd>=1.0.0,<3.0
@@ -17,7 +16,6 @@
 apscheduler>=3.0
 PrettyTable>=0.6,<0.8
 babel>=1.0
-six>=1.6.0
 ansible>=2.0.0.1
 kazoo
 sqlalchemy
diff --git a/tests/base.py b/tests/base.py
index bf6ad26..ff1f531 100755
--- a/tests/base.py
+++ b/tests/base.py
@@ -15,24 +15,20 @@
 # License for the specific language governing permissions and limitations
 # under the License.
 
-from six.moves import configparser as ConfigParser
+import configparser
 import datetime
 import gc
 import hashlib
+import importlib
+from io import StringIO
 import json
 import logging
 import os
-from six.moves import queue as Queue
-from six.moves import urllib
+import queue
 import random
 import re
 import select
 import shutil
-from six.moves import reload_module
-try:
-    from cStringIO import StringIO
-except Exception:
-    from six import StringIO
 import socket
 import string
 import subprocess
@@ -42,6 +38,7 @@
 import traceback
 import time
 import uuid
+import urllib
 
 
 import git
@@ -463,7 +460,7 @@
         super(FakeGerritConnection, self).__init__(driver, connection_name,
                                                    connection_config)
 
-        self.event_queue = Queue.Queue()
+        self.event_queue = queue.Queue()
         self.fixture_dir = os.path.join(FIXTURE_DIR, 'gerrit')
         self.change_number = 0
         self.changes = changes_db
@@ -936,7 +933,8 @@
                     'full_name': pr.project
                 }
             },
-            'files': pr.files
+            'files': pr.files,
+            'labels': pr.labels
         }
         return data
 
@@ -1373,8 +1371,8 @@
                                                 ssl_ca=ssl_ca)
 
     def getJobForConnection(self, connection, peek=False):
-        for queue in [self.high_queue, self.normal_queue, self.low_queue]:
-            for job in queue:
+        for job_queue in [self.high_queue, self.normal_queue, self.low_queue]:
+            for job in job_queue:
                 if not hasattr(job, 'waiting'):
                     if job.name.startswith(b'executor:execute'):
                         job.waiting = self.hold_jobs_in_queue
@@ -1384,7 +1382,7 @@
                     continue
                 if job.name in connection.functions:
                     if not peek:
-                        queue.remove(job)
+                        job_queue.remove(job)
                         connection.related_jobs[job.handle] = job
                         job.worker_connection = connection
                     job.running = True
@@ -1879,8 +1877,8 @@
         os.environ['STATSD_PORT'] = str(self.statsd.port)
         self.statsd.start()
         # the statsd client object is configured in the statsd module import
-        reload_module(statsd)
-        reload_module(zuul.scheduler)
+        importlib.reload(statsd)
+        importlib.reload(zuul.scheduler)
 
         self.gearman_server = FakeGearmanServer(self.use_ssl)
 
@@ -2008,7 +2006,7 @@
         # This creates the per-test configuration object.  It can be
         # overriden by subclasses, but should not need to be since it
         # obeys the config_file and tenant_config_file attributes.
-        self.config = ConfigParser.ConfigParser()
+        self.config = configparser.ConfigParser()
         self.config.read(os.path.join(FIXTURE_DIR, self.config_file))
 
         if not self.setupSimpleLayout():
@@ -2385,12 +2383,12 @@
         return True
 
     def eventQueuesEmpty(self):
-        for queue in self.event_queues:
-            yield queue.empty()
+        for event_queue in self.event_queues:
+            yield event_queue.empty()
 
     def eventQueuesJoin(self):
-        for queue in self.event_queues:
-            queue.join()
+        for event_queue in self.event_queues:
+            event_queue.join()
 
     def waitUntilSettled(self):
         self.log.debug("Waiting until settled...")
@@ -2399,8 +2397,9 @@
             if time.time() - start > self.wait_timeout:
                 self.log.error("Timeout waiting for Zuul to settle")
                 self.log.error("Queue status:")
-                for queue in self.event_queues:
-                    self.log.error("  %s: %s" % (queue, queue.empty()))
+                for event_queue in self.event_queues:
+                    self.log.error("  %s: %s" %
+                                   (event_queue, event_queue.empty()))
                 self.log.error("All builds waiting: %s" %
                                (self.areAllBuildsWaiting(),))
                 self.log.error("All builds reported: %s" %
@@ -2459,11 +2458,12 @@
         # Make sure there are no orphaned jobs
         for tenant in self.sched.abide.tenants.values():
             for pipeline in tenant.layout.pipelines.values():
-                for queue in pipeline.queues:
-                    if len(queue.queue) != 0:
+                for pipeline_queue in pipeline.queues:
+                    if len(pipeline_queue.queue) != 0:
                         print('pipeline %s queue %s contents %s' % (
-                            pipeline.name, queue.name, queue.queue))
-                    self.assertEqual(len(queue.queue), 0,
+                            pipeline.name, pipeline_queue.name,
+                            pipeline_queue.queue))
+                    self.assertEqual(len(pipeline_queue.queue), 0,
                                      "Pipelines queues should be empty")
 
     def assertReportedStat(self, key, value=None, kind=None):
diff --git a/tests/fixtures/layouts/requirements-github.yaml b/tests/fixtures/layouts/requirements-github.yaml
index 9933f27..891a366 100644
--- a/tests/fixtures/layouts/requirements-github.yaml
+++ b/tests/fixtures/layouts/requirements-github.yaml
@@ -168,6 +168,21 @@
       github:
         comment: true
 
+- pipeline:
+    name: require_label
+    manager: independent
+    require:
+      github:
+        label: approved
+    trigger:
+      github:
+        - event: pull_request
+          action: comment
+          comment: 'test me'
+    success:
+      github:
+        comment: true
+
 - job:
     name: project1-pipeline
 - job:
@@ -186,6 +201,8 @@
     name: project8-requireopen
 - job:
     name: project9-requirecurrent
+- job:
+    name: project10-label
 
 - project:
     name: org/project1
@@ -243,3 +260,9 @@
     require_current:
       jobs:
         - project9-requirecurrent
+
+- project:
+    name: org/project10
+    require_label:
+      jobs:
+        - project10-label
diff --git a/tests/unit/test_github_requirements.py b/tests/unit/test_github_requirements.py
index 43bdfc2..135f7ab 100644
--- a/tests/unit/test_github_requirements.py
+++ b/tests/unit/test_github_requirements.py
@@ -350,3 +350,28 @@
         self.waitUntilSettled()
         # Event hash is not current, should not trigger
         self.assertEqual(len(self.history), 1)
+
+    @simple_layout('layouts/requirements-github.yaml', driver='github')
+    def test_pipeline_require_label(self):
+        "Test pipeline requirement: label"
+        A = self.fake_github.openFakePullRequest('org/project10', 'master',
+                                                 'A')
+        # A comment event that we will keep submitting to trigger
+        comment = A.getCommentAddedEvent('test me')
+        self.fake_github.emitEvent(comment)
+        self.waitUntilSettled()
+        # No label so should not be enqueued
+        self.assertEqual(len(self.history), 0)
+
+        # A derp label should not cause it to be enqueued
+        A.addLabel('derp')
+        self.fake_github.emitEvent(comment)
+        self.waitUntilSettled()
+        self.assertEqual(len(self.history), 0)
+
+        # An approved label goes in
+        A.addLabel('approved')
+        self.fake_github.emitEvent(comment)
+        self.waitUntilSettled()
+        self.assertEqual(len(self.history), 1)
+        self.assertEqual(self.history[0].name, 'project10-label')
diff --git a/tests/unit/test_scheduler.py b/tests/unit/test_scheduler.py
index 2f9d272..eb17966 100755
--- a/tests/unit/test_scheduler.py
+++ b/tests/unit/test_scheduler.py
@@ -25,8 +25,8 @@
 from unittest import skip
 
 import git
-from six.moves import urllib
 import testtools
+import urllib
 
 import zuul.change_matcher
 from zuul.driver.gerrit import gerritreporter
diff --git a/tests/unit/test_webapp.py b/tests/unit/test_webapp.py
index b2836ae..da027c1 100644
--- a/tests/unit/test_webapp.py
+++ b/tests/unit/test_webapp.py
@@ -17,8 +17,8 @@
 
 import os
 import json
+import urllib
 
-from six.moves import urllib
 import webob
 
 from tests.base import ZuulTestCase, FIXTURE_DIR
diff --git a/tools/encrypt_secret.py b/tools/encrypt_secret.py
index 4865edd..e36b24e 100644
--- a/tools/encrypt_secret.py
+++ b/tools/encrypt_secret.py
@@ -17,7 +17,7 @@
 import subprocess
 import sys
 import tempfile
-from six.moves import urllib
+import urllib
 
 DESCRIPTION = """Encrypt a secret for Zuul.
 
diff --git a/zuul/ansible/callback/zuul_stream.py b/zuul/ansible/callback/zuul_stream.py
index 47e648e..e3d1e14 100644
--- a/zuul/ansible/callback/zuul_stream.py
+++ b/zuul/ansible/callback/zuul_stream.py
@@ -164,7 +164,7 @@
 
             hosts = self._get_task_hosts(task)
             for host in hosts:
-                if host in ('locahost', '127.0.0.1'):
+                if host in ('localhost', '127.0.0.1'):
                     # Don't try to stream from localhost
                     continue
                 ip = play_vars[host].get(
@@ -199,10 +199,10 @@
             self._stop_streamers()
         if result._task.action in ('command', 'shell'):
             stdout_lines = zuul_filter_result(result._result)
-        if is_localhost:
-            for line in stdout_lines:
-                ts, ln = (x.strip() for x in line.split(' | ', 1))
-                self._log("localhost | %s " % ln, ts=ts)
+            if is_localhost:
+                for line in stdout_lines:
+                    ts, ln = (x.strip() for x in line.split(' | ', 1))
+                    self._log("localhost | %s " % ln, ts=ts)
 
     def v2_runner_on_failed(self, result, ignore_errors=False):
         self._process_result_for_localhost(result)
diff --git a/zuul/cmd/__init__.py b/zuul/cmd/__init__.py
index d31c5b8..8610114 100755
--- a/zuul/cmd/__init__.py
+++ b/zuul/cmd/__init__.py
@@ -14,9 +14,9 @@
 # License for the specific language governing permissions and limitations
 # under the License.
 
-import six
-from six.moves import configparser as ConfigParser
+import configparser
 import extras
+import io
 import logging
 import logging.config
 import os
@@ -48,7 +48,7 @@
             yappi.start()
         else:
             yappi.stop()
-            yappi_out = six.BytesIO()
+            yappi_out = io.BytesIO()
             yappi.get_func_stats().print_all(out=yappi_out)
             yappi.get_thread_stats().print_all(out=yappi_out)
             log.debug(yappi_out.getvalue())
@@ -69,7 +69,7 @@
         return "Zuul version: %s" % zuul_version_info.release_string()
 
     def read_config(self):
-        self.config = ConfigParser.ConfigParser()
+        self.config = configparser.ConfigParser()
         if self.args.config:
             locations = [self.args.config]
         else:
diff --git a/zuul/configloader.py b/zuul/configloader.py
index 5e0fe65..d4f7c43 100644
--- a/zuul/configloader.py
+++ b/zuul/configloader.py
@@ -15,7 +15,6 @@
 import copy
 import os
 import logging
-import six
 import pprint
 import textwrap
 
@@ -427,7 +426,7 @@
                 setattr(job, a, conf[k])
         if 'nodes' in conf:
             conf_nodes = conf['nodes']
-            if isinstance(conf_nodes, six.string_types):
+            if isinstance(conf_nodes, str):
                 # This references an existing named nodeset in the layout.
                 ns = layout.nodesets[conf_nodes]
             else:
@@ -576,7 +575,7 @@
     def _parseJobList(tenant, layout, conf, source_context,
                       start_mark, job_list):
         for conf_job in conf:
-            if isinstance(conf_job, six.string_types):
+            if isinstance(conf_job, str):
                 attrs = dict(name=conf_job)
             elif isinstance(conf_job, dict):
                 # A dictionary in a job tree may override params
@@ -1007,7 +1006,7 @@
 
     @staticmethod
     def _getProject(source, conf, current_include):
-        if isinstance(conf, six.string_types):
+        if isinstance(conf, str):
             # Return a project object whether conf is a dict or a str
             project = source.getProject(conf)
             project_include = current_include
@@ -1031,7 +1030,7 @@
     def _getProjects(source, conf, current_include):
         # Return a project object whether conf is a dict or a str
         projects = []
-        if isinstance(conf, six.string_types):
+        if isinstance(conf, str):
             # A simple project name string
             projects.append(TenantParser._getProject(
                 source, conf, current_include))
diff --git a/zuul/connection/__init__.py b/zuul/connection/__init__.py
index 90ab39c..3655115 100644
--- a/zuul/connection/__init__.py
+++ b/zuul/connection/__init__.py
@@ -15,11 +15,9 @@
 import abc
 
 import extras
-import six
 
 
-@six.add_metaclass(abc.ABCMeta)
-class BaseConnection(object):
+class BaseConnection(object, metaclass=abc.ABCMeta):
     """Base class for connections.
 
     A connection is a shared object that sources, triggers and reporters can
diff --git a/zuul/driver/__init__.py b/zuul/driver/__init__.py
index bb28d1f..c78283d 100644
--- a/zuul/driver/__init__.py
+++ b/zuul/driver/__init__.py
@@ -14,11 +14,8 @@
 
 import abc
 
-import six
 
-
-@six.add_metaclass(abc.ABCMeta)
-class Driver(object):
+class Driver(object, metaclass=abc.ABCMeta):
     """A Driver is an extension component of Zuul that supports
     interfacing with a remote system.  It can support any of the following
     interfaces (but must support at least one to be useful):
@@ -80,8 +77,7 @@
         pass
 
 
-@six.add_metaclass(abc.ABCMeta)
-class ConnectionInterface(object):
+class ConnectionInterface(object, metaclass=abc.ABCMeta):
     """The Connection interface.
 
     A driver which is able to supply a Connection should implement
@@ -124,8 +120,7 @@
         pass
 
 
-@six.add_metaclass(abc.ABCMeta)
-class TriggerInterface(object):
+class TriggerInterface(object, metaclass=abc.ABCMeta):
     """The trigger interface.
 
     A driver which is able to supply a trigger should implement this
@@ -167,8 +162,7 @@
         pass
 
 
-@six.add_metaclass(abc.ABCMeta)
-class SourceInterface(object):
+class SourceInterface(object, metaclass=abc.ABCMeta):
     """The source interface to be implemented by a driver.
 
     A driver which is able to supply a Source should implement this
@@ -216,8 +210,7 @@
         pass
 
 
-@six.add_metaclass(abc.ABCMeta)
-class ReporterInterface(object):
+class ReporterInterface(object, metaclass=abc.ABCMeta):
     """The reporter interface to be implemented by a driver.
 
     A driver which is able to supply a Reporter should implement this
@@ -256,8 +249,7 @@
         pass
 
 
-@six.add_metaclass(abc.ABCMeta)
-class WrapperInterface(object):
+class WrapperInterface(object, metaclass=abc.ABCMeta):
     """The wrapper interface to be implmeneted by a driver.
 
     A driver which wraps execution of commands executed by Zuul should
diff --git a/zuul/driver/bubblewrap/__init__.py b/zuul/driver/bubblewrap/__init__.py
index b92b365..9e9a26e 100644
--- a/zuul/driver/bubblewrap/__init__.py
+++ b/zuul/driver/bubblewrap/__init__.py
@@ -19,11 +19,10 @@
 import logging
 import os
 import pwd
+import shlex
 import subprocess
 import sys
 
-from six.moves import shlex_quote
-
 from zuul.driver import (Driver, WrapperInterface)
 
 
@@ -134,6 +133,7 @@
             ['{}'.format(x).encode('utf8') for x in passwd])
         (passwd_r, passwd_w) = os.pipe()
         os.write(passwd_w, passwd_bytes)
+        os.write(passwd_w, b'\n')
         os.close(passwd_w)
 
         gid = os.getgid()
@@ -142,6 +142,7 @@
             ['{}'.format(x).encode('utf8') for x in group])
         group_r, group_w = os.pipe()
         os.write(group_w, group_bytes)
+        os.write(group_w, b'\n')
         os.close(group_w)
 
         kwargs = dict(kwargs)  # Don't update passed in dict
@@ -153,7 +154,7 @@
         command = [x.format(**kwargs) for x in bwrap_command]
 
         self.log.debug("Bubblewrap command: %s",
-                       " ".join(shlex_quote(c) for c in command))
+                       " ".join(shlex.quote(c) for c in command))
 
         wrapped_popen = WrappedPopen(command, passwd_r, group_r)
 
diff --git a/zuul/driver/gerrit/gerritconnection.py b/zuul/driver/gerrit/gerritconnection.py
index 9033ee6..39a81bc 100644
--- a/zuul/driver/gerrit/gerritconnection.py
+++ b/zuul/driver/gerrit/gerritconnection.py
@@ -18,11 +18,11 @@
 import select
 import threading
 import time
-from six.moves import queue as Queue
-from six.moves import shlex_quote
 import paramiko
 import logging
 import pprint
+import shlex
+import queue
 import voluptuous as v
 
 from zuul.connection import BaseConnection
@@ -260,7 +260,7 @@
         self.keyfile = self.connection_config.get('sshkey', None)
         self.keepalive = int(self.connection_config.get('keepalive', 60))
         self.watcher_thread = None
-        self.event_queue = Queue.Queue()
+        self.event_queue = queue.Queue()
         self.client = None
 
         self.baseurl = self.connection_config.get('baseurl',
@@ -606,7 +606,7 @@
     def review(self, project, change, message, action={}):
         cmd = 'gerrit review --project %s' % project
         if message:
-            cmd += ' --message %s' % shlex_quote(message)
+            cmd += ' --message %s' % shlex.quote(message)
         for key, val in action.items():
             if val is True:
                 cmd += ' --%s' % key
diff --git a/zuul/driver/git/gitconnection.py b/zuul/driver/git/gitconnection.py
index ca88d3f..f4fe7e5 100644
--- a/zuul/driver/git/gitconnection.py
+++ b/zuul/driver/git/gitconnection.py
@@ -14,7 +14,7 @@
 # under the License.
 
 import logging
-from six.moves import urllib
+import urllib
 
 import voluptuous as v
 
diff --git a/zuul/driver/github/githubconnection.py b/zuul/driver/github/githubconnection.py
index f9f1c27..7a3491e 100644
--- a/zuul/driver/github/githubconnection.py
+++ b/zuul/driver/github/githubconnection.py
@@ -524,6 +524,7 @@
                                            change.patchset)
         change.reviews = self.getPullReviews(change.project,
                                              change.number)
+        change.labels = change.pr.get('labels')
 
         return change
 
@@ -572,8 +573,11 @@
             self.log.warning("Pull request #%s of %s/%s returned None!" % (
                              number, owner, proj))
             time.sleep(1)
+        # Get the issue obj so we can get the labels (this is silly)
+        issueobj = probj.issue()
         pr = probj.as_dict()
         pr['files'] = [f.filename for f in probj.files()]
+        pr['labels'] = [l.name for l in issueobj.labels()]
         log_rate_limit(self.log, github)
         return pr
 
@@ -599,7 +603,7 @@
             if not pr_url:
                 continue
             # the issue provides no good description of the project :\
-            owner, project, _, number = pr_url.split('/')[4:]
+            owner, project, _, number = pr_url.split('/')[-4:]
             github = self.getGithubClient("%s/%s" % (owner, project))
             pr = github.pull_request(owner, project, number)
             if pr.head.sha != sha:
diff --git a/zuul/driver/github/githubmodel.py b/zuul/driver/github/githubmodel.py
index cfd1bc0..db119f0 100644
--- a/zuul/driver/github/githubmodel.py
+++ b/zuul/driver/github/githubmodel.py
@@ -28,9 +28,13 @@
 class PullRequest(Change):
     def __init__(self, project):
         super(PullRequest, self).__init__(project)
+        self.project = None
+        self.pr = None
         self.updated_at = None
         self.title = None
         self.reviews = []
+        self.files = []
+        self.labels = []
 
     def isUpdateOf(self, other):
         if (hasattr(other, 'number') and self.number == other.number and
@@ -284,7 +288,8 @@
 
 class GithubRefFilter(RefFilter, GithubCommonFilter):
     def __init__(self, connection_name, statuses=[], required_reviews=[],
-                 reject_reviews=[], open=None, current_patchset=None):
+                 reject_reviews=[], open=None, current_patchset=None,
+                 labels=[]):
         RefFilter.__init__(self, connection_name)
 
         GithubCommonFilter.__init__(self, required_reviews=required_reviews,
@@ -293,6 +298,7 @@
         self.statuses = statuses
         self.open = open
         self.current_patchset = current_patchset
+        self.labels = labels
 
     def __repr__(self):
         ret = '<GithubRefFilter'
@@ -310,6 +316,8 @@
             ret += ' open: %s' % self.open
         if self.current_patchset:
             ret += ' current-patchset: %s' % self.current_patchset
+        if self.labels:
+            ret += ' labels: %s' % self.labels
 
         ret += '>'
 
@@ -341,4 +349,9 @@
         if not self.matchesReviews(change):
             return False
 
+        # required labels are ANDed
+        for label in self.labels:
+            if label not in change.labels:
+                return False
+
         return True
diff --git a/zuul/driver/github/githubsource.py b/zuul/driver/github/githubsource.py
index 519ebf1..1bd280f 100644
--- a/zuul/driver/github/githubsource.py
+++ b/zuul/driver/github/githubsource.py
@@ -97,6 +97,7 @@
             required_reviews=to_list(config.get('review')),
             open=config.get('open'),
             current_patchset=config.get('current-patchset'),
+            labels=to_list(config.get('label')),
         )
         return [f]
 
@@ -121,7 +122,8 @@
     require = {'status': scalar_or_list(str),
                'review': scalar_or_list(review),
                'open': bool,
-               'current-patchset': bool}
+               'current-patchset': bool,
+               'label': scalar_or_list(str)}
     return require
 
 
diff --git a/zuul/executor/server.py b/zuul/executor/server.py
index 7256aaf..657a063 100644
--- a/zuul/executor/server.py
+++ b/zuul/executor/server.py
@@ -18,6 +18,7 @@
 import os
 import shutil
 import signal
+import shlex
 import socket
 import subprocess
 import tempfile
@@ -28,7 +29,6 @@
 from zuul.lib.config import get_default
 
 import gear
-from six.moves import shlex_quote
 
 import zuul.merger.merger
 import zuul.ansible
@@ -1263,7 +1263,7 @@
             if self.aborted:
                 return (self.RESULT_ABORTED, None)
             self.log.debug("Ansible command: ANSIBLE_CONFIG=%s %s",
-                           config_file, " ".join(shlex_quote(c) for c in cmd))
+                           config_file, " ".join(shlex.quote(c) for c in cmd))
             self.proc = popen(
                 cmd,
                 cwd=self.jobdir.work_root,
diff --git a/zuul/lib/clonemapper.py b/zuul/lib/clonemapper.py
index 57ac177..7423308 100644
--- a/zuul/lib/clonemapper.py
+++ b/zuul/lib/clonemapper.py
@@ -14,17 +14,11 @@
 # under the License.
 
 from collections import defaultdict
-import extras
+from collections import OrderedDict
 import logging
 import os
 import re
 
-import six
-
-
-OrderedDict = extras.try_imports(['collections.OrderedDict',
-                                  'ordereddict.OrderedDict'])
-
 
 class CloneMapper(object):
     log = logging.getLogger("zuul.CloneMapper")
@@ -62,17 +56,17 @@
             raise Exception("Expansion error. Check error messages above")
 
         self.log.info("Mapping projects to workspace...")
-        for project, dest in six.iteritems(ret):
+        for project, dest in ret.items():
             dest = os.path.normpath(os.path.join(workspace, dest[0]))
             ret[project] = dest
             self.log.info("  %s -> %s", project, dest)
 
         self.log.debug("Checking overlap in destination directories...")
         check = defaultdict(list)
-        for project, dest in six.iteritems(ret):
+        for project, dest in ret.items():
             check[dest].append(project)
 
-        dupes = dict((d, p) for (d, p) in six.iteritems(check) if len(p) > 1)
+        dupes = dict((d, p) for (d, p) in check.items() if len(p) > 1)
         if dupes:
             raise Exception("Some projects share the same destination: %s",
                             dupes)
diff --git a/zuul/lib/cloner.py b/zuul/lib/cloner.py
index 3070be6..3fcffbe 100644
--- a/zuul/lib/cloner.py
+++ b/zuul/lib/cloner.py
@@ -18,8 +18,6 @@
 import os
 import re
 
-import six
-
 from git import GitCommandError
 from zuul import exceptions
 from zuul.lib.clonemapper import CloneMapper
@@ -72,7 +70,7 @@
         dests = mapper.expand(workspace=self.workspace)
 
         self.log.info("Preparing %s repositories", len(dests))
-        for project, dest in six.iteritems(dests):
+        for project, dest in dests.items():
             self.prepareRepo(project, dest)
         self.log.info("Prepared all repositories")
 
diff --git a/zuul/lib/commandsocket.py b/zuul/lib/commandsocket.py
index ae62204..901291a 100644
--- a/zuul/lib/commandsocket.py
+++ b/zuul/lib/commandsocket.py
@@ -18,7 +18,7 @@
 import os
 import socket
 import threading
-from six.moves import queue
+import queue
 
 
 class CommandSocket(object):
diff --git a/zuul/model.py b/zuul/model.py
index 610b9b6..a89c6d1 100644
--- a/zuul/model.py
+++ b/zuul/model.py
@@ -13,19 +13,13 @@
 # under the License.
 
 import abc
+from collections import OrderedDict
 import copy
 import logging
 import os
 import struct
 import time
 from uuid import uuid4
-import extras
-
-import six
-
-OrderedDict = extras.try_imports(['collections.OrderedDict',
-                                  'ordereddict.OrderedDict'])
-
 
 MERGER_MERGE = 1          # "git merge"
 MERGER_MERGE_RESOLVE = 2  # "git merge -s resolve"
@@ -669,8 +663,7 @@
             path=self.path)
 
 
-@six.add_metaclass(abc.ABCMeta)
-class Role(object):
+class Role(object, metaclass=abc.ABCMeta):
     """A reference to an ansible role."""
 
     def __init__(self, target_name):
diff --git a/zuul/reporter/__init__.py b/zuul/reporter/__init__.py
index 89830d5..0ac5766 100644
--- a/zuul/reporter/__init__.py
+++ b/zuul/reporter/__init__.py
@@ -15,11 +15,8 @@
 import abc
 import logging
 
-import six
 
-
-@six.add_metaclass(abc.ABCMeta)
-class BaseReporter(object):
+class BaseReporter(object, metaclass=abc.ABCMeta):
     """Base class for reporters.
 
     Defines the exact public methods that must be supplied.
diff --git a/zuul/rpclistener.py b/zuul/rpclistener.py
index 5474c8a..be3b7d1 100644
--- a/zuul/rpclistener.py
+++ b/zuul/rpclistener.py
@@ -19,7 +19,6 @@
 import traceback
 
 import gear
-import six
 
 from zuul import model
 from zuul.lib.config import get_default
@@ -168,8 +167,7 @@
         # TODO: use args to filter by pipeline etc
         running_items = []
         for tenant in self.sched.abide.tenants.values():
-            for pipeline_name, pipeline in six.iteritems(
-                    tenant.layout.pipelines):
+            for pipeline_name, pipeline in tenant.layout.pipelines.items():
                 for queue in pipeline.queues:
                     for item in queue.queue:
                         running_items.append(item.formatJSON())
diff --git a/zuul/scheduler.py b/zuul/scheduler.py
index 098cf87..c762309 100644
--- a/zuul/scheduler.py
+++ b/zuul/scheduler.py
@@ -20,8 +20,7 @@
 import logging
 import os
 import pickle
-import six
-from six.moves import queue as Queue
+import queue
 import socket
 import sys
 import threading
@@ -50,7 +49,9 @@
     def wait(self, timeout=None):
         self._wait_event.wait(timeout)
         if self._exc_info:
-            six.reraise(*self._exc_info)
+            # http://python3porting.com/differences.html#raise
+            e, v, t = self._exc_info
+            raise e(v).with_traceback(t)
         return self._wait_event.is_set()
 
 
@@ -218,9 +219,9 @@
         self.triggers = dict()
         self.config = config
 
-        self.trigger_event_queue = Queue.Queue()
-        self.result_event_queue = Queue.Queue()
-        self.management_event_queue = Queue.Queue()
+        self.trigger_event_queue = queue.Queue()
+        self.result_event_queue = queue.Queue()
+        self.management_event_queue = queue.Queue()
         self.abide = model.Abide()
 
         if not testonly:
diff --git a/zuul/source/__init__.py b/zuul/source/__init__.py
index 68baf0e..b37aeb4 100644
--- a/zuul/source/__init__.py
+++ b/zuul/source/__init__.py
@@ -14,11 +14,8 @@
 
 import abc
 
-import six
 
-
-@six.add_metaclass(abc.ABCMeta)
-class BaseSource(object):
+class BaseSource(object, metaclass=abc.ABCMeta):
     """Base class for sources.
 
     A source class gives methods for fetching and updating changes. Each
diff --git a/zuul/trigger/__init__.py b/zuul/trigger/__init__.py
index a5406d6..a67c99b 100644
--- a/zuul/trigger/__init__.py
+++ b/zuul/trigger/__init__.py
@@ -14,11 +14,8 @@
 
 import abc
 
-import six
 
-
-@six.add_metaclass(abc.ABCMeta)
-class BaseTrigger(object):
+class BaseTrigger(object, metaclass=abc.ABCMeta):
     """Base class for triggers.
 
     Defines the exact public methods that must be supplied."""