You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by se...@apache.org on 2016/11/08 22:29:06 UTC

aurora git commit: Resolve docker tags to concrete identifiers for DockerContainerizer

Repository: aurora
Updated Branches:
  refs/heads/master 84bde070f -> 793711f56


Resolve docker tags to concrete identifiers for DockerContainerizer

Docker tags are mutable and can point to different different images
at different points in time. This makes a job launched with a Docker
image to be mutable across restarts of the job. This breaks Aurora's
guarantee of job immutability (except via job updates).

This change introduces a binding helper, that resolves docker name:tag
to a concrete registry/name@digest identifier. Identifying
docker images via a content-addressable digest is available via the
Docker Registry v2, that is a prerequisite for this feature.

Bugs closed: AURORA-1014

Reviewed at https://reviews.apache.org/r/52479/


Project: http://git-wip-us.apache.org/repos/asf/aurora/repo
Commit: http://git-wip-us.apache.org/repos/asf/aurora/commit/793711f5
Tree: http://git-wip-us.apache.org/repos/asf/aurora/tree/793711f5
Diff: http://git-wip-us.apache.org/repos/asf/aurora/diff/793711f5

Branch: refs/heads/master
Commit: 793711f56d83d532240cc5ca39404ff504b14430
Parents: 84bde07
Author: Santhosh Kumar Shanmugham <sa...@gmail.com>
Authored: Tue Nov 8 23:27:05 2016 +0100
Committer: Stephan Erb <se...@apache.org>
Committed: Tue Nov 8 23:27:05 2016 +0100

----------------------------------------------------------------------
 3rdparty/python/requirements.txt                |   1 +
 RELEASE-NOTES.md                                |   5 +
 Vagrantfile                                     |   2 +-
 build-support/packer/build.sh                   |   1 +
 docs/features/containers.md                     |  26 ++-
 docs/reference/client-cluster-configuration.md  |   6 +
 examples/jobs/hello_docker_engine.aurora        |   8 +
 .../aurora/client/binding_helpers/__init__.py   |  13 ++
 .../client/binding_helpers/docker_helper.py     |  43 +++++
 .../python/apache/aurora/client/cli/client.py   |  19 +++
 .../apache/aurora/client/docker/__init__.py     |  13 ++
 .../aurora/client/docker/docker_client.py       | 137 ++++++++++++++++
 .../aurora/client/binding_helpers/__init__.py   |  13 ++
 .../binding_helpers/test_docker_helper.py       |  84 ++++++++++
 .../python/apache/aurora/client/docker/BUILD    |  24 +++
 .../apache/aurora/client/docker/__init__.py     |  13 ++
 .../aurora/client/docker/test_docker_client.py  | 163 +++++++++++++++++++
 src/test/python/apache/aurora/config/BUILD      |   2 +
 .../apache/aurora/e2e/http/http_example.aurora  |   2 +-
 .../http/http_example_bad_healthcheck.aurora    |   2 +-
 .../aurora/e2e/http/http_example_updated.aurora |   2 +-
 .../sh/org/apache/aurora/e2e/test_end_to_end.sh |  19 ++-
 22 files changed, 589 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/793711f5/3rdparty/python/requirements.txt
----------------------------------------------------------------------
diff --git a/3rdparty/python/requirements.txt b/3rdparty/python/requirements.txt
index 2fb6f4c..df2db7a 100644
--- a/3rdparty/python/requirements.txt
+++ b/3rdparty/python/requirements.txt
@@ -25,6 +25,7 @@ psutil==4.3.0
 pystachio==0.8.3
 requests==2.11.1
 requests-kerberos==0.10.0
+requests-mock==1.1.0
 subprocess32==3.2.7
 thrift==0.9.1
 twitter.common.app==0.3.7

http://git-wip-us.apache.org/repos/asf/aurora/blob/793711f5/RELEASE-NOTES.md
----------------------------------------------------------------------
diff --git a/RELEASE-NOTES.md b/RELEASE-NOTES.md
index 374bee6..3924962 100644
--- a/RELEASE-NOTES.md
+++ b/RELEASE-NOTES.md
@@ -11,6 +11,11 @@
   Containerizer if the scheduler is running with the `-allow_container_volumes`
   flag.
 * The executor will send SIGTERM to processes that self daemonize via double forking.
+- Resolve docker tags to concrete identifiers for DockerContainerizer, so that job configuration
+  is immutable across restarts. The feature introduces a new `{{docker.image[name][tag]}}` binder that
+  can be used in the Aurora job configuration to resolve a docker image specified by its `name:tag`
+  to a concrete identifier specified by its `registry/name@digest`. It requires version 2 of the
+  Docker Registry.
 
 ### Deprecations and removals:
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/793711f5/Vagrantfile
----------------------------------------------------------------------
diff --git a/Vagrantfile b/Vagrantfile
index 08131c9..7b05e20 100644
--- a/Vagrantfile
+++ b/Vagrantfile
@@ -25,7 +25,7 @@ Vagrant.configure(VAGRANTFILE_API_VERSION) do |config|
   config.vm.hostname = "aurora.local"
   # See build-support/packer/README.md for instructions on updating this box.
   config.vm.box = "apache-aurora/dev-environment"
-  config.vm.box_version = "0.0.9"
+  config.vm.box_version = "0.0.10"
 
   config.vm.define "devcluster" do |dev|
     dev.vm.network :private_network, ip: "192.168.33.7", :auto_config => false

http://git-wip-us.apache.org/repos/asf/aurora/blob/793711f5/build-support/packer/build.sh
----------------------------------------------------------------------
diff --git a/build-support/packer/build.sh b/build-support/packer/build.sh
index f5157a6..2fb4723 100644
--- a/build-support/packer/build.sh
+++ b/build-support/packer/build.sh
@@ -58,6 +58,7 @@ function install_docker {
     linux-image-extra-$(uname -r) \
     apparmor \
     docker-engine
+  docker run -d -p 5000:5000 --restart=always --name registry registry:2
 }
 
 function install_docker2aci {

http://git-wip-us.apache.org/repos/asf/aurora/blob/793711f5/docs/features/containers.md
----------------------------------------------------------------------
diff --git a/docs/features/containers.md b/docs/features/containers.md
index 8af38e3..32b99fe 100644
--- a/docs/features/containers.md
+++ b/docs/features/containers.md
@@ -66,7 +66,12 @@ Docker Containerizer
 
 The Docker containerizer launches container images using the Docker engine. It may often provide
 more advanced features than the native Mesos containerizer, but has to be installed separately to
-Mesos on each agent host,
+Mesos on each agent host.
+
+Starting with the 0.17.0 release, `image` can be specified with a `{{docker.image[name][tag]}}` binder so that
+the tag can be resolved to a concrete image digest. This ensures that the job always uses the same image
+across restarts, even if the version identified by the tag has been updated, guaranteeing that only job
+updates can mutate configuration.
 
 Example (available in the [Vagrant environment](../getting-started/vagrant.md)):
 
@@ -93,9 +98,28 @@ Example (available in the [Vagrant environment](../getting-started/vagrant.md)):
         name = 'hello_docker',
         task = task,
         container = Docker(image = 'python:2.7')
+      ), Service(
+        cluster = 'devcluster',
+        environment = 'devel',
+        role = 'www-data',
+        name = 'hello_docker_engine_binding',
+        task = task,
+        container = Docker(image = '{{docker.image[library/python][2.7]}}')
       )
     ]
 
+Note, this feature requires a v2 Docker registry. If using a private Docker registry its url
+must be specified in the `clusters.json` configuration file under the key `docker_registry`.
+If not specified `docker_registry` defaults to `https://registry-1.docker.io` (Docker Hub).
+
+Example:
+    # clusters.json
+    [{
+      "name": "devcluster",
+      ...
+      "docker_registry": "https://registry.example.com"
+    }]
+
 Details of how to use Docker via the Docker engine can be found in the
 [Reference Documentation](../reference/configuration.md#docker-object). Please note that in order to
 correctly execute processes inside a job, the Docker container must have Python 2.7 and potentitally

http://git-wip-us.apache.org/repos/asf/aurora/blob/793711f5/docs/reference/client-cluster-configuration.md
----------------------------------------------------------------------
diff --git a/docs/reference/client-cluster-configuration.md b/docs/reference/client-cluster-configuration.md
index 5a86cda..6a62673 100644
--- a/docs/reference/client-cluster-configuration.md
+++ b/docs/reference/client-cluster-configuration.md
@@ -35,6 +35,7 @@ The following properties may be set:
    **scheduler_uri**       | String   | URI of Aurora scheduler instance.
    **proxy_url**           | String   | Used by the client to format URLs for display.
    **auth_mechanism**      | String   | The authentication mechanism to use when communicating with the scheduler. (Default: UNAUTHENTICATED)
+   **docker_registry**     | String   | Used by the client to resolve docker tags.
 
 
 ## Details
@@ -91,3 +92,8 @@ URL of your VIP in a loadbalancer or a roundrobin DNS name.
 The identifier of an authentication mechanism that the client should use when communicating with the
 scheduler. Support for values other than `UNAUTHENTICATED` requires a matching scheduler-side
 [security configuration](../operations/security.md).
+
+### `docker_registry`
+
+The URI of the Docker Registry that will be used by the Aurora client to resolve docker tags to concrete
+image ids, when using the docker binding helper, like `{{docker.image[name][tag]}}`.

http://git-wip-us.apache.org/repos/asf/aurora/blob/793711f5/examples/jobs/hello_docker_engine.aurora
----------------------------------------------------------------------
diff --git a/examples/jobs/hello_docker_engine.aurora b/examples/jobs/hello_docker_engine.aurora
index 3c830e8..99d99a2 100644
--- a/examples/jobs/hello_docker_engine.aurora
+++ b/examples/jobs/hello_docker_engine.aurora
@@ -35,5 +35,13 @@ jobs = [
     name = 'hello_docker_engine',
     task = task,
     container = Docker(image = 'python:2.7')
+  ),
+  Service(
+    cluster = 'devcluster',
+    environment = 'devel',
+    role = 'www-data',
+    name = 'hello_docker_engine_binding',
+    task = task,
+    container = Docker(image = '{{docker.image[library/python][2.7]}}')
   )
 ]

http://git-wip-us.apache.org/repos/asf/aurora/blob/793711f5/src/main/python/apache/aurora/client/binding_helpers/__init__.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/binding_helpers/__init__.py b/src/main/python/apache/aurora/client/binding_helpers/__init__.py
new file mode 100644
index 0000000..0663a9a
--- /dev/null
+++ b/src/main/python/apache/aurora/client/binding_helpers/__init__.py
@@ -0,0 +1,13 @@
+#
+# 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.
+#

http://git-wip-us.apache.org/repos/asf/aurora/blob/793711f5/src/main/python/apache/aurora/client/binding_helpers/docker_helper.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/binding_helpers/docker_helper.py b/src/main/python/apache/aurora/client/binding_helpers/docker_helper.py
new file mode 100644
index 0000000..92d5db3
--- /dev/null
+++ b/src/main/python/apache/aurora/client/binding_helpers/docker_helper.py
@@ -0,0 +1,43 @@
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+from pystachio import Ref
+from pystachio.matcher import Any, Matcher
+
+from apache.aurora.client.binding_helper import BindingHelper
+from apache.aurora.client.docker.docker_client import DockerRegistryClient
+from apache.aurora.common.clusters import CLUSTERS
+
+
+class DockerBindingHelper(BindingHelper):
+  @property
+  def name(self):
+    return 'docker'
+
+  @property
+  def matcher(self):
+    return Matcher('docker').image[Any][Any]
+
+  def bind(self, config, match, env, binding_dict):
+    cluster = CLUSTERS[config.cluster()]
+    image = match
+    ref_str = 'docker.image[%s][%s]' % image[2:4]
+    ref = Ref.from_address(ref_str)
+    if ref_str in binding_dict:
+      (image_data, image_struct) = binding_dict[ref_str]
+    else:
+      image_data = '%s:%s' % (image[2], image[3])
+      image_struct = DockerRegistryClient.resolve(cluster, image[2], image[3])
+    binding_dict[ref_str] = (image_data, image_struct)
+    config.bind({ref: image_struct})

http://git-wip-us.apache.org/repos/asf/aurora/blob/793711f5/src/main/python/apache/aurora/client/cli/client.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/client.py b/src/main/python/apache/aurora/client/cli/client.py
index fa0c264..03020e5 100644
--- a/src/main/python/apache/aurora/client/cli/client.py
+++ b/src/main/python/apache/aurora/client/cli/client.py
@@ -18,6 +18,8 @@ import sys
 
 from twitter.common.log.formatters.plain import PlainFormatter
 
+from apache.aurora.client.binding_helper import BindingHelper
+from apache.aurora.client.binding_helpers.docker_helper import DockerBindingHelper
 from apache.aurora.client.cli import CommandLine, ConfigurationPlugin
 from apache.aurora.client.cli.options import CommandOption
 from apache.aurora.common.auth.auth_module_manager import register_auth_module
@@ -87,6 +89,22 @@ class AuroraAuthConfigurationPlugin(ConfigurationPlugin):
     pass
 
 
+class AuroraHelpersPlugin(ConfigurationPlugin):
+  """Plugin for configuring binding helpers."""
+
+  def get_options(self):
+    return []
+
+  def before_dispatch(self, raw_args):
+    return raw_args
+
+  def before_execution(self, context):
+    BindingHelper.register(DockerBindingHelper())
+
+  def after_execution(self, context, result_code):
+    pass
+
+
 class AuroraCommandLine(CommandLine):
   """The CommandLine implementation for the Aurora client command line."""
 
@@ -94,6 +112,7 @@ class AuroraCommandLine(CommandLine):
     super(AuroraCommandLine, self).__init__()
     self.register_plugin(AuroraLogConfigurationPlugin())
     self.register_plugin(AuroraAuthConfigurationPlugin())
+    self.register_plugin(AuroraHelpersPlugin())
 
   @property
   def name(self):

http://git-wip-us.apache.org/repos/asf/aurora/blob/793711f5/src/main/python/apache/aurora/client/docker/__init__.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/docker/__init__.py b/src/main/python/apache/aurora/client/docker/__init__.py
new file mode 100644
index 0000000..0663a9a
--- /dev/null
+++ b/src/main/python/apache/aurora/client/docker/__init__.py
@@ -0,0 +1,13 @@
+#
+# 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.
+#

http://git-wip-us.apache.org/repos/asf/aurora/blob/793711f5/src/main/python/apache/aurora/client/docker/docker_client.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/docker/docker_client.py b/src/main/python/apache/aurora/client/docker/docker_client.py
new file mode 100644
index 0000000..f4b5007
--- /dev/null
+++ b/src/main/python/apache/aurora/client/docker/docker_client.py
@@ -0,0 +1,137 @@
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+from urlparse import urlparse
+
+import requests
+from pystachio import Default, String
+from twitter.common import log
+
+from apache.aurora.common.cluster import Cluster
+
+ACCEPT_HEADER = {'Accept': 'application/vnd.docker.distribution.manifest.v2+json'}
+MANIFESTS_URL = '%s/v2/%s/manifests/%s'
+DEFAULT_DOCKER_REGISTRY_HOST = 'https://registry-1.docker.io'
+
+
+class DockerRegistryTrait(Cluster.Trait):
+  docker_registry = Default(String, DEFAULT_DOCKER_REGISTRY_HOST)   # noqa
+
+
+class DockerClientException(Exception):
+  pass
+
+
+class DockerRegistryClient(object):
+  @classmethod
+  def _get_auth_token(cls, www_authenticate, headers=None):
+    # solve the auth challenge
+    args = {}
+    for t in www_authenticate.split(' ')[1].split(','):
+      k, v = t.replace('"', '').split('=')
+      if k == 'realm':
+        domain = v
+      elif k != 'error':
+        args[k] = v
+
+    if not domain:
+      raise DockerClientException('Unable to determine auth endpoint. %s' % headers)
+
+    response = requests.get(domain, params=args, headers=headers)
+    response.raise_for_status()
+    if response.status_code == requests.codes.ok and 'token' in response.json():
+      return response.json()['token']
+    else:
+      raise DockerClientException('Failed to get auth token. %s' % response.text)
+
+  @classmethod
+  def _get_auth_challenge(cls, response):
+    if 'Www-Authenticate' not in response.headers:
+      raise DockerClientException('No auth challenge. Www-Authenticate header not returned. %s'
+        % response.headers)
+
+    return response.headers.get('Www-Authenticate')
+
+  @classmethod
+  def _solve_auth_challenge(cls, response, registry):
+    www_authenticate = cls._get_auth_challenge(response)
+    challenge_type = www_authenticate.split(' ')[0]
+    if 'Bearer' == challenge_type:
+      return {'Authorization': 'Bearer %s' % cls._get_auth_token(www_authenticate)}
+    elif 'Basic' == challenge_type:
+      raise DockerClientException('Basic authorization not supported: %s' % www_authenticate)
+    else:
+      raise DockerClientException('Unknown auth challenge: %s' % www_authenticate)
+
+  @classmethod
+  def _resolve_image(cls, registry, name, tag, headers=None):
+    url = MANIFESTS_URL % (registry, name, tag)
+    response = requests.head(url, headers=headers)
+
+    if response.status_code == requests.codes.unauthorized:
+      # solve the auth challenge and retry again
+      authorization = cls._solve_auth_challenge(response, registry)
+      if headers is None:
+        headers = dict()
+      headers.update(authorization)
+      response = requests.head(url, headers=headers)
+
+      if response.status_code == requests.codes.unauthorized:
+        # its a private repo, raise exception
+        raise DockerClientException('Private Docker repository - %s:%s' % (name, tag))
+
+    if response.status_code == requests.codes.ok:
+      image_ref = '%s@%s' % (name, response.headers.get('Docker-Content-Digest'))
+
+      if registry != DEFAULT_DOCKER_REGISTRY_HOST:
+        image_ref = '%s/%s' % (urlparse(registry).netloc, image_ref)
+
+      log.info('Resolved %s:%s => %s' % (name, tag, image_ref))
+      return image_ref
+
+    # something is wrong
+    response.raise_for_status()
+    raise DockerClientException('Unable to resolve image %s:%s' % (name, tag))
+
+  @classmethod
+  def _version_check(cls, registry):
+    response = requests.head('%s/v2/' % registry)
+
+    if response.status_code == requests.codes.unauthorized:
+      # retry request with token received by solving auth challenge
+      response = requests.head('%s/v2/' % registry,
+        headers=cls._solve_auth_challenge(response, registry))
+
+    if response.status_code == requests.codes.ok:
+      return True
+
+    # something is wrong
+    response.raise_for_status()
+    return False
+
+  @classmethod
+  def resolve(cls, cluster, name, tag='latest'):
+    """
+      Docker image reference:
+        [REGISTRY_HOST[:REGISTRY_PORT]/]REPOSITORY[:TAG|@DIGEST]
+      See https://github.com/docker/distribution/blob/master/reference/reference.go
+    """
+
+    # Image digests is only supported in registry v2 - https://github.com/docker/docker/pull/11109
+    # Check the registry and confirm that it is version 2.
+    cluster = cluster.with_trait(DockerRegistryTrait)
+    if cls._version_check(cluster.docker_registry):
+      return cls._resolve_image(cluster.docker_registry, name, tag, headers=ACCEPT_HEADER)
+    else:
+      raise DockerClientException('%s is not a registry v2' % cluster.docker_registry)

http://git-wip-us.apache.org/repos/asf/aurora/blob/793711f5/src/test/python/apache/aurora/client/binding_helpers/__init__.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/binding_helpers/__init__.py b/src/test/python/apache/aurora/client/binding_helpers/__init__.py
new file mode 100644
index 0000000..0663a9a
--- /dev/null
+++ b/src/test/python/apache/aurora/client/binding_helpers/__init__.py
@@ -0,0 +1,13 @@
+#
+# 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.
+#

http://git-wip-us.apache.org/repos/asf/aurora/blob/793711f5/src/test/python/apache/aurora/client/binding_helpers/test_docker_helper.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/binding_helpers/test_docker_helper.py b/src/test/python/apache/aurora/client/binding_helpers/test_docker_helper.py
new file mode 100644
index 0000000..b019f90
--- /dev/null
+++ b/src/test/python/apache/aurora/client/binding_helpers/test_docker_helper.py
@@ -0,0 +1,84 @@
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+import unittest
+
+import pytest
+from mock import call, patch
+from twitter.common.contextutil import temporary_file
+
+from apache.aurora.client import binding_helper
+from apache.aurora.client.binding_helper import BindingHelper
+from apache.aurora.client.binding_helpers.docker_helper import DockerBindingHelper
+from apache.aurora.client.docker.docker_client import DockerRegistryClient
+from apache.aurora.common.cluster import Cluster
+from apache.aurora.common.clusters import CLUSTERS
+from apache.aurora.config import AuroraConfig
+
+DOCKER_BINDING_CONFIG = """
+jobs = [Job(
+  name = 'hello_world',
+  role = 'john_doe',
+  environment = 'staging42',
+  cluster = 'smf1-test',
+  task = Task(
+    name = 'main',
+    processes = [Process(name='command', cmdline="echo helloworld")],
+    resources = Resources(cpu = 0.1, ram = 64 * MB, disk = 64 * MB),
+  ),
+  container = Docker(image='{{docker.image[some/name][some.tag]}}')
+)]
+"""
+
+TEST_CLUSTER = Cluster(
+    name='smf1-test',
+    docker_registry='some.registry.domain.com')
+TEST_CLUSTERS = [TEST_CLUSTER]
+
+
+class TestDockerBindingHelper(unittest.TestCase):
+
+  @patch.object(DockerRegistryClient, 'resolve')
+  def test_docker_binding(self, mock_resolve):
+    image_reference = 'registry.example.com/some/repo@some:digest'
+
+    mock_resolve.return_value = image_reference
+
+    binding_helper.unregister_all()
+    BindingHelper.register(DockerBindingHelper())
+
+    with temporary_file() as fp:
+      fp.write(DOCKER_BINDING_CONFIG)
+      fp.flush()
+      with CLUSTERS.patch(TEST_CLUSTERS):
+        cfg = AuroraConfig.load(fp.name)
+        binding_helper.apply_all(cfg)
+        assert cfg.job().taskConfig.container.docker.image == image_reference
+        assert mock_resolve.mock_calls == [call(TEST_CLUSTER, 'some/name', 'some.tag')]
+
+  @patch.object(DockerRegistryClient, 'resolve')
+  def test_docker_binding_throws(self, mock_resolve):
+    mock_resolve.side_effect = Exception('mock resolve failure')
+
+    binding_helper.unregister_all()
+    BindingHelper.register(DockerBindingHelper())
+
+    with temporary_file() as fp:
+      fp.write(DOCKER_BINDING_CONFIG)
+      fp.flush()
+      with CLUSTERS.patch(TEST_CLUSTERS):
+        cfg = AuroraConfig.load(fp.name)
+        with pytest.raises(Exception):
+          binding_helper.apply_all(cfg)
+          assert mock_resolve.mock_calls == [call(TEST_CLUSTER, 'some/name', 'some.tag')]

http://git-wip-us.apache.org/repos/asf/aurora/blob/793711f5/src/test/python/apache/aurora/client/docker/BUILD
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/docker/BUILD b/src/test/python/apache/aurora/client/docker/BUILD
new file mode 100644
index 0000000..5ea2cf7
--- /dev/null
+++ b/src/test/python/apache/aurora/client/docker/BUILD
@@ -0,0 +1,24 @@
+#
+# 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.
+#
+
+python_tests(
+  name = 'docker',
+  sources = globs('*py'),
+  dependencies = [
+    '3rdparty/python:requests',
+    '3rdparty/python:requests-mock',
+    '3rdparty/python:twitter.common.contextutil',
+    'src/main/python/apache/aurora/client'
+  ]
+)

http://git-wip-us.apache.org/repos/asf/aurora/blob/793711f5/src/test/python/apache/aurora/client/docker/__init__.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/docker/__init__.py b/src/test/python/apache/aurora/client/docker/__init__.py
new file mode 100644
index 0000000..0663a9a
--- /dev/null
+++ b/src/test/python/apache/aurora/client/docker/__init__.py
@@ -0,0 +1,13 @@
+#
+# 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.
+#

http://git-wip-us.apache.org/repos/asf/aurora/blob/793711f5/src/test/python/apache/aurora/client/docker/test_docker_client.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/docker/test_docker_client.py b/src/test/python/apache/aurora/client/docker/test_docker_client.py
new file mode 100644
index 0000000..4efc350
--- /dev/null
+++ b/src/test/python/apache/aurora/client/docker/test_docker_client.py
@@ -0,0 +1,163 @@
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+from urlparse import urlparse
+
+import pytest
+import requests
+import requests_mock
+
+from apache.aurora.client.docker.docker_client import (
+    ACCEPT_HEADER,
+    DEFAULT_DOCKER_REGISTRY_HOST,
+    MANIFESTS_URL,
+    DockerClientException,
+    DockerRegistryClient,
+    DockerRegistryTrait
+)
+from apache.aurora.common.clusters import Cluster
+
+REGISTRY_URL = 'https://registry.example.com'
+NAME = 'library/ubuntu'
+TAG = 'latest'
+IMAGE_DIGEST = 'sha256:45bc58500fa3d3c0d67233d4a7798134b46b486af1389ca87000c543f46c3d24'
+EXPECTED_REFERENCE = '%s/%s@%s' % (urlparse(REGISTRY_URL).netloc, NAME, IMAGE_DIGEST)
+
+
+def test_docker_registry_trait_default():
+  cluster = Cluster().with_trait(DockerRegistryTrait)
+  assert cluster.docker_registry == DEFAULT_DOCKER_REGISTRY_HOST
+
+
+def test_docker_binding_version_check_failure():
+  do_test_docker_binding_failure(version_success=False)
+
+
+def test_docker_binding_tag_not_found():
+  do_test_docker_binding_failure(manifests_success=False)
+
+
+def test_docker_binding_auth_token_failure():
+  do_test_docker_binding_failure(auth_success=False)
+
+
+def do_test_docker_binding_failure(version_success=True, manifests_success=True,
+  auth_success=True):
+  with requests_mock.Mocker() as m:
+    mock_registry(m, need_auth_token=True, version_success=version_success,
+      manifests_success=manifests_success, auth_success=auth_success)
+
+    with pytest.raises(DockerClientException):
+      DockerRegistryClient.resolve(mock_cluster(), NAME, TAG)
+
+
+def test_docker_binding_unknown_registry():
+  with pytest.raises(Exception):
+    DockerRegistryClient.resolve(mock_cluster(), NAME, TAG)
+
+
+def test_docker_binding_public_repo():
+  do_test_docker_binding_success()
+
+
+def test_docker_binding_with_auth_public_repo():
+  do_test_docker_binding_success(need_auth_token=True)
+
+
+def do_test_docker_binding_success(need_auth_token=False):
+  with requests_mock.Mocker() as m:
+    mock_registry(m, need_auth_token=need_auth_token)
+    assert DockerRegistryClient.resolve(mock_cluster(), NAME, TAG) == EXPECTED_REFERENCE
+
+
+def mock_version(request, context):
+  context.status_code = requests.codes.ok
+  return {}
+
+
+def mock_version_not_supported(request, context):
+  context.status_code = requests.codes.moved_permanently
+  return {'error': 'version not found'}
+
+
+def mock_version_with_auth(request, context):
+  if request.headers.get('Authorization') == 'Bearer some-token':
+    return mock_version(request, context)
+  else:
+    context.status_code = requests.codes.unauthorized
+    context.headers.update(
+      {'Www-Authenticate': 'Bearer realm="https://auth.docker.io/token",'
+      + 'service="registry.docker.io"'})
+    return {'error': 'unauthorized'}
+
+
+def mock_manifests(request, context):
+  if request.headers['Accept'] == ACCEPT_HEADER['Accept']:
+    context.status_code = requests.codes.ok
+    context.headers.update({'Docker-Content-Digest': IMAGE_DIGEST})
+    return {}
+  else:
+    raise Exception("Missing Accept header")
+
+
+def mock_manifests_tag_failure(request, context):
+  context.status_code = requests.codes.moved_permanently
+  return {'error': 'tag not found'}
+
+
+def mock_manifests_with_auth(request, context):
+  if request.headers.get('Authorization') == 'Bearer some-token':
+    return mock_manifests(request, context)
+  else:
+    context.status_code = requests.codes.unauthorized
+    context.headers.update(
+      {'Www-Authenticate': 'Bearer realm="https://auth.docker.io/token",'
+      + 'service="registry.docker.io",scope="repository:library/ubuntu:*"'})
+    return {'error': 'unauthorized'}
+
+
+def mock_token(request, context):
+  return {'token': 'some-token'}
+
+
+def mock_token_failure(request, context):
+  context.status_code = requests.codes.moved_permanently
+  return {'error': 'token not found'}
+
+
+def mock_token_with_creds(request, context):
+  if request.headers.get('Authorization') == 'Basic some-auth':
+    return mock_token(request, context)
+  else:
+    return {'token': 'wrong-token'}
+
+
+def mock_registry(m, need_auth_token=False, auth_success=True,
+  version_success=True, manifests_success=True):
+  if need_auth_token:
+    m.register_uri('HEAD', '%s/v2/' % REGISTRY_URL,
+      [{'json': mock_version_with_auth if version_success else mock_version_not_supported}])
+    m.register_uri('GET', 'https://auth.docker.io/token?service=registry.docker.io',
+      [{'json': mock_token if auth_success else mock_token_failure}])
+    m.register_uri('HEAD', MANIFESTS_URL % (REGISTRY_URL, NAME, TAG),
+        [{'json': mock_manifests_with_auth if manifests_success else mock_manifests_tag_failure}])
+  else:
+    m.register_uri('HEAD', '%s/v2/' % REGISTRY_URL,
+      [{'json': mock_version if version_success else mock_version_not_supported}])
+    m.register_uri('HEAD', MANIFESTS_URL % (REGISTRY_URL, NAME, TAG),
+        [{'json': mock_manifests if manifests_success else mock_manifests_tag_failure}])
+
+
+def mock_cluster(docker_registry=REGISTRY_URL):
+  return Cluster(name='smf1-test', docker_registry=docker_registry)

http://git-wip-us.apache.org/repos/asf/aurora/blob/793711f5/src/test/python/apache/aurora/config/BUILD
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/config/BUILD b/src/test/python/apache/aurora/config/BUILD
index 52b6040..ec7909d 100644
--- a/src/test/python/apache/aurora/config/BUILD
+++ b/src/test/python/apache/aurora/config/BUILD
@@ -16,9 +16,11 @@ python_tests(
   name = 'config',
   sources = globs('*py'),
   dependencies = [
+    '3rdparty/python:requests-mock',
     '3rdparty/python:twitter.common.contextutil',
     'api/src/main/thrift/org/apache/aurora/gen',
     'src/main/python/apache/aurora/config',
+    'src/main/python/apache/aurora/client',
   ],
   coverage = [
     'apache.aurora.config',

http://git-wip-us.apache.org/repos/asf/aurora/blob/793711f5/src/test/sh/org/apache/aurora/e2e/http/http_example.aurora
----------------------------------------------------------------------
diff --git a/src/test/sh/org/apache/aurora/e2e/http/http_example.aurora b/src/test/sh/org/apache/aurora/e2e/http/http_example.aurora
index c71fb81..c9dae28 100644
--- a/src/test/sh/org/apache/aurora/e2e/http/http_example.aurora
+++ b/src/test/sh/org/apache/aurora/e2e/http/http_example.aurora
@@ -104,7 +104,7 @@ jobs = [
   ).bind(profile=DefaultProfile()),
   job(
     name = 'http_example_docker',
-    container = Container(docker=Docker(image = 'http_example'))
+    container = Docker(image='{{docker.image[http_example][latest]}}'),
   ).bind(profile=ContainerProfile),
   job(
     name = 'http_example_gpu'

http://git-wip-us.apache.org/repos/asf/aurora/blob/793711f5/src/test/sh/org/apache/aurora/e2e/http/http_example_bad_healthcheck.aurora
----------------------------------------------------------------------
diff --git a/src/test/sh/org/apache/aurora/e2e/http/http_example_bad_healthcheck.aurora b/src/test/sh/org/apache/aurora/e2e/http/http_example_bad_healthcheck.aurora
index 4fa387d..b85ace4 100644
--- a/src/test/sh/org/apache/aurora/e2e/http/http_example_bad_healthcheck.aurora
+++ b/src/test/sh/org/apache/aurora/e2e/http/http_example_bad_healthcheck.aurora
@@ -86,7 +86,7 @@ jobs = [
   ).bind(profile=DefaultProfile()),
   job(
     name = 'http_example_docker',
-    container = Container(docker=Docker(image = 'http_example'))
+    container = Docker(image='{{docker.image[http_example][latest]}}'),
   ).bind(profile=ContainerProfile),
   job(
     name = 'http_example_gpu'

http://git-wip-us.apache.org/repos/asf/aurora/blob/793711f5/src/test/sh/org/apache/aurora/e2e/http/http_example_updated.aurora
----------------------------------------------------------------------
diff --git a/src/test/sh/org/apache/aurora/e2e/http/http_example_updated.aurora b/src/test/sh/org/apache/aurora/e2e/http/http_example_updated.aurora
index b279b4f..b3caa41 100644
--- a/src/test/sh/org/apache/aurora/e2e/http/http_example_updated.aurora
+++ b/src/test/sh/org/apache/aurora/e2e/http/http_example_updated.aurora
@@ -72,7 +72,7 @@ jobs = [
   ).bind(profile=DefaultProfile()),
   job(
     name = 'http_example_docker',
-    container = Container(docker=Docker(image = 'http_example'))
+    container = Docker(image='{{docker.image[http_example][latest]}}'),
   ).bind(profile=ContainerProfile),
   job(
     name = 'http_example_gpu'

http://git-wip-us.apache.org/repos/asf/aurora/blob/793711f5/src/test/sh/org/apache/aurora/e2e/test_end_to_end.sh
----------------------------------------------------------------------
diff --git a/src/test/sh/org/apache/aurora/e2e/test_end_to_end.sh b/src/test/sh/org/apache/aurora/e2e/test_end_to_end.sh
index f014b28..9cc6cec 100755
--- a/src/test/sh/org/apache/aurora/e2e/test_end_to_end.sh
+++ b/src/test/sh/org/apache/aurora/e2e/test_end_to_end.sh
@@ -37,6 +37,8 @@ tear_down() {
     aurora update abort devcluster/vagrant/test/$job || true >/dev/null 2>&1
     aurora job killall --no-batching devcluster/vagrant/test/$job >/dev/null 2>&1
   done
+
+  sudo mv /etc/aurora/clusters.json.old /etc/aurora/clusters.json || true > /dev/null 2>&1
 }
 
 collect_result() {
@@ -45,9 +47,9 @@ collect_result() {
     echo "OK (all tests passed)"
   else
     echo "!!! FAIL (something returned non-zero) for $BASH_COMMAND"
-    # Attempt to clean up any state we left behind.
-    tear_down
   fi
+  # Attempt to clean up any state we left behind.
+  tear_down
   exit $RETCODE
 }
 
@@ -453,6 +455,16 @@ setup_image_stores() {
   rm -rf "$TEMP_PATH"
 }
 
+setup_docker_registry() {
+  # build the test docker image
+  sudo docker build -t http_example -f "${TEST_ROOT}/Dockerfile.python" ${TEST_ROOT}
+  docker tag http_example:latest aurora.local:5000/http_example:latest
+  docker login -p testpassword -u testuser http://aurora.local:5000
+  docker push aurora.local:5000/http_example:latest
+  sudo mv /etc/aurora/clusters.json /etc/aurora/clusters.json.old
+  sudo sh -c "cat /etc/aurora/clusters.json.old | jq 'map(. + {docker_registry:\"http://aurora.local:5000\"})' > /etc/aurora/clusters.json"
+}
+
 test_appc_unified() {
   num_mounts_before=$(mount |wc -l |tr -d '\n')
 
@@ -536,6 +548,7 @@ trap collect_result EXIT
 
 aurorabuild all
 setup_ssh
+setup_docker_registry
 
 test_version
 test_http_example "${TEST_JOB_ARGS[@]}"
@@ -545,8 +558,6 @@ test_http_example_basic "${TEST_JOB_REVOCABLE_ARGS[@]}"
 
 test_http_example_basic "${TEST_JOB_GPU_ARGS[@]}"
 
-# build the test docker image
-sudo docker build -t http_example -f "${TEST_ROOT}/Dockerfile.python" ${TEST_ROOT}
 test_http_example "${TEST_JOB_DOCKER_ARGS[@]}"
 
 setup_image_stores