You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by ma...@apache.org on 2015/04/11 03:03:06 UTC

aurora git commit: Adding client Kerberos support.

Repository: aurora
Updated Branches:
  refs/heads/master 084a62258 -> eb9e40888


Adding client Kerberos support.

Bugs closed: AURORA-813

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


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

Branch: refs/heads/master
Commit: eb9e408887c9aa57c44bf17fc2c4cd5aa9c00960
Parents: 084a622
Author: Maxim Khutornenko <ma...@apache.org>
Authored: Fri Apr 10 18:02:09 2015 -0700
Committer: Maxim Khutornenko <ma...@apache.org>
Committed: Fri Apr 10 18:02:09 2015 -0700

----------------------------------------------------------------------
 3rdparty/python/requirements.txt                |  1 +
 examples/vagrant/aurorabuild.sh                 |  8 +--
 examples/vagrant/clusters_kerberos.json         |  8 +++
 .../upstart/aurora-scheduler-kerberos.conf      |  4 +-
 src/main/python/apache/aurora/admin/BUILD       | 13 +++-
 .../python/apache/aurora/admin/aurora_admin.py  |  7 ++
 .../python/apache/aurora/client/api/__init__.py |  2 +-
 .../aurora/client/api/scheduler_client.py       | 18 ++++--
 src/main/python/apache/aurora/client/cli/BUILD  | 10 +++
 .../python/apache/aurora/client/cli/client.py   | 23 +++++++
 src/main/python/apache/aurora/common/auth/BUILD | 15 ++++-
 .../apache/aurora/common/auth/__init__.py       | 11 ----
 .../apache/aurora/common/auth/auth_module.py    | 19 +++++-
 .../aurora/common/auth/auth_module_manager.py   | 16 ++++-
 .../apache/aurora/common/auth/kerberos.py       | 39 +++++++++++
 .../python/apache/aurora/common/transport.py    | 12 +++-
 .../aurora/client/api/test_scheduler_client.py  | 68 ++++++++++++++++++--
 .../apache/aurora/common/test_transport.py      | 30 ++++++++-
 .../aurora/e2e/test_kerberos_end_to_end.sh      | 55 ++++++++++++----
 19 files changed, 307 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/eb9e4088/3rdparty/python/requirements.txt
----------------------------------------------------------------------
diff --git a/3rdparty/python/requirements.txt b/3rdparty/python/requirements.txt
index 11a307c..c23f98c 100644
--- a/3rdparty/python/requirements.txt
+++ b/3rdparty/python/requirements.txt
@@ -10,6 +10,7 @@ protobuf==2.6.1
 psutil==2.1.3
 pystachio==0.7.4
 requests==2.3.0
+requests-kerberos==0.6.1
 thrift==0.9.1
 twitter.common.app==0.3.3
 twitter.common.collections==0.3.3

http://git-wip-us.apache.org/repos/asf/aurora/blob/eb9e4088/examples/vagrant/aurorabuild.sh
----------------------------------------------------------------------
diff --git a/examples/vagrant/aurorabuild.sh b/examples/vagrant/aurorabuild.sh
index f11488a..1a76e4f 100755
--- a/examples/vagrant/aurorabuild.sh
+++ b/examples/vagrant/aurorabuild.sh
@@ -33,13 +33,13 @@ function upstart_update {
 }
 
 function build_client {
-  ./pants binary src/main/python/apache/aurora/client/cli:aurora
-  sudo ln -sf $DIST_DIR/aurora.pex /usr/local/bin/aurora
+  ./pants binary src/main/python/apache/aurora/client/cli:kaurora
+  sudo ln -sf $DIST_DIR/kaurora.pex /usr/local/bin/aurora
 }
 
 function build_admin_client {
-  ./pants binary src/main/python/apache/aurora/admin:aurora_admin
-  sudo ln -sf $DIST_DIR/aurora_admin.pex /usr/local/bin/aurora_admin
+  ./pants binary src/main/python/apache/aurora/admin:kaurora_admin
+  sudo ln -sf $DIST_DIR/kaurora_admin.pex /usr/local/bin/aurora_admin
 }
 
 function build_scheduler {

http://git-wip-us.apache.org/repos/asf/aurora/blob/eb9e4088/examples/vagrant/clusters_kerberos.json
----------------------------------------------------------------------
diff --git a/examples/vagrant/clusters_kerberos.json b/examples/vagrant/clusters_kerberos.json
new file mode 100644
index 0000000..f19e33b
--- /dev/null
+++ b/examples/vagrant/clusters_kerberos.json
@@ -0,0 +1,8 @@
+[{
+  "name": "devcluster",
+  "zk": "192.168.33.7",
+  "scheduler_zk_path": "/aurora/scheduler",
+  "auth_mechanism": "KERBEROS",
+  "slave_run_directory": "latest",
+  "slave_root": "/var/lib/mesos"
+}]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/aurora/blob/eb9e4088/examples/vagrant/upstart/aurora-scheduler-kerberos.conf
----------------------------------------------------------------------
diff --git a/examples/vagrant/upstart/aurora-scheduler-kerberos.conf b/examples/vagrant/upstart/aurora-scheduler-kerberos.conf
index 0a809e8..7736ff4 100644
--- a/examples/vagrant/upstart/aurora-scheduler-kerberos.conf
+++ b/examples/vagrant/upstart/aurora-scheduler-kerberos.conf
@@ -52,5 +52,5 @@ exec $DIST_DIR/install/aurora-scheduler/bin/aurora-scheduler \
   -shiro_realm_modules=org.apache.aurora.scheduler.http.api.security.Kerberos5ShiroRealmModule,org.apache.aurora.scheduler.http.api.security.IniShiroRealmModule \
   -shiro_ini_path=/home/vagrant/aurora/src/test/resources/org/apache/aurora/scheduler/http/api/security/shiro-example.ini \
   -http_authentication_mechanism=NEGOTIATE \
-  -kerberos_server_keytab=/home/vagrant/krb5-1.13.1/build/testdir/HTTP-localhost.keytab \
-  -kerberos_server_principal=HTTP/localhost@KRBTEST.COM
+  -kerberos_server_keytab=/home/vagrant/krb5-1.13.1/build/testdir/HTTP-192.168.33.7.keytab \
+  -kerberos_server_principal=HTTP/192.168.33.7@KRBTEST.COM

http://git-wip-us.apache.org/repos/asf/aurora/blob/eb9e4088/src/main/python/apache/aurora/admin/BUILD
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/admin/BUILD b/src/main/python/apache/aurora/admin/BUILD
index c51a90b..22bf3f9 100644
--- a/src/main/python/apache/aurora/admin/BUILD
+++ b/src/main/python/apache/aurora/admin/BUILD
@@ -24,7 +24,7 @@ python_library(
     '3rdparty/python:twitter.common.quantity',
     'src/main/python/apache/aurora/client/api',
     'src/main/python/apache/aurora/client:base',
-    'src/main/python/apache/aurora/common:clusters',
+    'src/main/python/apache/aurora/common',
     'src/main/python/apache/aurora/client:config',
     'api/src/main/thrift/org/apache/aurora/gen:py-thrift',
   ]
@@ -81,6 +81,16 @@ python_binary(
   ]
 )
 
+# TODO(maxim): consider making this default ("aurora_admin") target along with kerberos auth.
+python_binary(
+  name = 'kaurora_admin',
+  entry_point = 'apache.aurora.admin.aurora_admin:proxy_main',
+  dependencies = [
+    ':aurora_admin_lib',
+    'src/main/python/apache/aurora/common/auth:kerberos',
+  ],
+)
+
 python_library(
   name = 'aurora_admin_lib',
   sources = [ 'aurora_admin.py' ],
@@ -94,7 +104,6 @@ python_library(
   name = 'admin-packaged',
   dependencies = [
     ':aurora_admin_lib',
-    'src/main/python/apache/aurora/common',
     'src/main/python/apache/aurora/config:config-packaged',
   ],
   provides = setup_py(

http://git-wip-us.apache.org/repos/asf/aurora/blob/eb9e4088/src/main/python/apache/aurora/admin/aurora_admin.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/admin/aurora_admin.py b/src/main/python/apache/aurora/admin/aurora_admin.py
index f9e8f3d..1c39717 100644
--- a/src/main/python/apache/aurora/admin/aurora_admin.py
+++ b/src/main/python/apache/aurora/admin/aurora_admin.py
@@ -17,6 +17,7 @@ from twitter.common.log.options import LogOptions
 
 from apache.aurora.admin import help as help_commands
 from apache.aurora.admin import admin, maintenance
+from apache.aurora.common.auth.auth_module_manager import register_auth_module
 
 from .help import add_verbosity_options, generate_terse_usage
 
@@ -27,6 +28,12 @@ add_verbosity_options()
 def main():
   app.help()
 
+try:
+  from apache.aurora.common.auth.kerberos import KerberosAuthModule
+  register_auth_module(KerberosAuthModule())
+except ImportError:
+  # Use default auth implementation if kerberos is not available.
+  pass
 
 LogOptions.set_stderr_log_level('INFO')
 LogOptions.disable_disk_logging()

http://git-wip-us.apache.org/repos/asf/aurora/blob/eb9e4088/src/main/python/apache/aurora/client/api/__init__.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/api/__init__.py b/src/main/python/apache/aurora/client/api/__init__.py
index a81329f..0ae1d9f 100644
--- a/src/main/python/apache/aurora/client/api/__init__.py
+++ b/src/main/python/apache/aurora/client/api/__init__.py
@@ -18,7 +18,7 @@ from twitter.common import log
 
 from apache.aurora.client.base import combine_messages
 from apache.aurora.common.aurora_job_key import AuroraJobKey
-from apache.aurora.common.auth import make_session_key
+from apache.aurora.common.auth.auth_module_manager import make_session_key
 from apache.aurora.common.cluster import Cluster
 
 from .restarter import Restarter

http://git-wip-us.apache.org/repos/asf/aurora/blob/eb9e4088/src/main/python/apache/aurora/client/api/scheduler_client.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/api/scheduler_client.py b/src/main/python/apache/aurora/client/api/scheduler_client.py
index 95e5534..3f9c691 100644
--- a/src/main/python/apache/aurora/client/api/scheduler_client.py
+++ b/src/main/python/apache/aurora/client/api/scheduler_client.py
@@ -26,7 +26,11 @@ from twitter.common.quantity import Amount, Time
 from twitter.common.zookeeper.kazoo_client import TwitterKazooClient
 from twitter.common.zookeeper.serverset import ServerSet
 
-from apache.aurora.common.auth import make_session_key, SessionKeyError
+from apache.aurora.common.auth.auth_module_manager import (
+    get_auth_handler,
+    make_session_key,
+    SessionKeyError
+)
 from apache.aurora.common.cluster import Cluster
 from apache.aurora.common.transport import TRequestsTransport
 
@@ -60,19 +64,21 @@ class SchedulerClient(object):
   #   ZookeeperClientTrait
   #   DirectClientTrait
   @classmethod
-  def get(cls, cluster, **kwargs):
+  def get(cls, cluster, auth_factory=get_auth_handler, **kwargs):
     if not isinstance(cluster, Cluster):
       raise TypeError('"cluster" must be an instance of Cluster, got %s' % type(cluster))
     cluster = cluster.with_trait(SchedulerClientTrait)
+    auth_handler = auth_factory(cluster.auth_mechanism)
     if cluster.zk:
-      return ZookeeperSchedulerClient(cluster, port=cluster.zk_port, **kwargs)
+      return ZookeeperSchedulerClient(cluster, port=cluster.zk_port, auth=auth_handler, **kwargs)
     elif cluster.scheduler_uri:
-      return DirectSchedulerClient(cluster.scheduler_uri, **kwargs)
+      return DirectSchedulerClient(cluster.scheduler_uri, auth=auth_handler, **kwargs)
     else:
       raise ValueError('"cluster" does not specify zk or scheduler_uri')
 
-  def __init__(self, user_agent, verbose=False):
+  def __init__(self, auth, user_agent, verbose=False):
     self._client = None
+    self._auth = auth
     self._user_agent = user_agent
     self._verbose = verbose
 
@@ -87,7 +93,7 @@ class SchedulerClient(object):
     return None
 
   def _connect_scheduler(self, uri, clock=time):
-    transport = TRequestsTransport(uri, user_agent=self._user_agent)
+    transport = TRequestsTransport(uri, auth=self._auth, user_agent=self._user_agent)
     protocol = TJSONProtocol.TJSONProtocol(transport)
     schedulerClient = AuroraAdmin.Client(protocol)
     for _ in range(self.THRIFT_RETRIES):

http://git-wip-us.apache.org/repos/asf/aurora/blob/eb9e4088/src/main/python/apache/aurora/client/cli/BUILD
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/BUILD b/src/main/python/apache/aurora/client/cli/BUILD
index 3ab41cc..973d059 100644
--- a/src/main/python/apache/aurora/client/cli/BUILD
+++ b/src/main/python/apache/aurora/client/cli/BUILD
@@ -20,6 +20,16 @@ python_binary(
   ],
 )
 
+# TODO(maxim): consider making this default ("aurora") target along with kerberos auth.
+python_binary(
+  name='kaurora',
+  entry_point = 'apache.aurora.client.cli.client:proxy_main',
+  dependencies = [
+    ':client_lib',
+    'src/main/python/apache/aurora/common/auth:kerberos',
+  ],
+)
+
 python_library(
   name = 'client_lib',
   sources = [ 'client.py' ],

http://git-wip-us.apache.org/repos/asf/aurora/blob/eb9e4088/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 24516d1..c0974f3 100644
--- a/src/main/python/apache/aurora/client/cli/client.py
+++ b/src/main/python/apache/aurora/client/cli/client.py
@@ -20,6 +20,7 @@ from twitter.common.log.formatters.plain import PlainFormatter
 
 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
 
 
 class AuroraLogConfigurationPlugin(ConfigurationPlugin):
@@ -54,12 +55,34 @@ class AuroraLogConfigurationPlugin(ConfigurationPlugin):
     pass
 
 
+class AuroraAuthConfigurationPlugin(ConfigurationPlugin):
+  """Plugin for configuring aurora client authentication."""
+
+  def get_options(self):
+    return []
+
+  def before_dispatch(self, raw_args):
+    return raw_args
+
+  def before_execution(self, context):
+    try:
+      from apache.aurora.common.auth.kerberos import KerberosAuthModule
+      register_auth_module(KerberosAuthModule())
+    except ImportError:
+      # Use default auth implementation if kerberos is not available.
+      pass
+
+  def after_execution(self, context, result_code):
+    pass
+
+
 class AuroraCommandLine(CommandLine):
   """The CommandLine implementation for the Aurora client command line."""
 
   def __init__(self):
     super(AuroraCommandLine, self).__init__()
     self.register_plugin(AuroraLogConfigurationPlugin())
+    self.register_plugin(AuroraAuthConfigurationPlugin())
 
   @property
   def name(self):

http://git-wip-us.apache.org/repos/asf/aurora/blob/eb9e4088/src/main/python/apache/aurora/common/auth/BUILD
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/common/auth/BUILD b/src/main/python/apache/aurora/common/auth/BUILD
index 9664846..0abac94 100644
--- a/src/main/python/apache/aurora/common/auth/BUILD
+++ b/src/main/python/apache/aurora/common/auth/BUILD
@@ -14,10 +14,23 @@
 
 python_library(
   name = 'auth',
-  sources = globs('*.py'),
+  sources = [
+    '__init__.py',
+    'auth_module.py',
+    'auth_module_manager.py',
+  ],
   dependencies = [
     '3rdparty/python:twitter.common.lang',
     '3rdparty/python:twitter.common.log',
     'api/src/main/thrift/org/apache/aurora/gen:py-thrift',
   ]
 )
+
+python_library(
+  name = 'kerberos',
+  sources = ['kerberos.py'],
+  dependencies = [
+     '3rdparty/python:requests-kerberos',
+     ':auth'
+  ]
+)

http://git-wip-us.apache.org/repos/asf/aurora/blob/eb9e4088/src/main/python/apache/aurora/common/auth/__init__.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/common/auth/__init__.py b/src/main/python/apache/aurora/common/auth/__init__.py
index 3119fd6..0663a9a 100644
--- a/src/main/python/apache/aurora/common/auth/__init__.py
+++ b/src/main/python/apache/aurora/common/auth/__init__.py
@@ -11,14 +11,3 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-
-from .auth_module import AuthModule, InsecureAuthModule  # noqa
-from .auth_module_manager import make_session_key, register_auth_module, SessionKeyError  # noqa
-
-__all__ = (
-    'AuthModule',
-    'InsecureAuthModule',
-    'make_session_key',
-    'register_auth_module',
-    'SessionKeyError',
-)

http://git-wip-us.apache.org/repos/asf/aurora/blob/eb9e4088/src/main/python/apache/aurora/common/auth/auth_module.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/common/auth/auth_module.py b/src/main/python/apache/aurora/common/auth/auth_module.py
index 5f4116e..53a3182 100644
--- a/src/main/python/apache/aurora/common/auth/auth_module.py
+++ b/src/main/python/apache/aurora/common/auth/auth_module.py
@@ -22,15 +22,27 @@ from gen.apache.aurora.api.ttypes import SessionKey
 class AuthModule(Interface):
   @abstractproperty
   def mechanism(self):
-    """Return the mechanism provided by this AuthModule."""
+    """Return the mechanism provided by this AuthModule.
+    ":rtype: string
+    """
 
+  # TODO(maxim): drop this in AURORA-1229
   @abstractmethod
   def payload(self):
-    """Return the payload generated by the AuthModule."""
+    """Return the payload generated by the AuthModule.
+    ":rtype: binary blob data.
+    """
 
+  # TODO(maxim): drop this or replace with self.auth() in AURORA-1229
   def __call__(self):
     return SessionKey(mechanism=self.mechanism, data=self.payload())
 
+  @abstractmethod
+  def auth(self):
+    """Authentication handler for the HTTP transport layer.
+    :rtype: requests.auth.AuthBase.
+    """
+
 
 class InsecureAuthModule(AuthModule):
   @property
@@ -39,3 +51,6 @@ class InsecureAuthModule(AuthModule):
 
   def payload(self):
     return 'UNAUTHENTICATED'
+
+  def auth(self):
+    return None

http://git-wip-us.apache.org/repos/asf/aurora/blob/eb9e4088/src/main/python/apache/aurora/common/auth/auth_module_manager.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/common/auth/auth_module_manager.py b/src/main/python/apache/aurora/common/auth/auth_module_manager.py
index 73a8e5c..73c6bd7 100644
--- a/src/main/python/apache/aurora/common/auth/auth_module_manager.py
+++ b/src/main/python/apache/aurora/common/auth/auth_module_manager.py
@@ -20,8 +20,9 @@ from gen.apache.aurora.api.ttypes import SessionKey
 
 _INSECURE_AUTH_MODULE = InsecureAuthModule()
 _AUTH_MODULES = {
-  _INSECURE_AUTH_MODULE.mechanism: _INSECURE_AUTH_MODULE
+  _INSECURE_AUTH_MODULE.mechanism: _INSECURE_AUTH_MODULE,
 }
+DEFAULT_AUTH_MECHANISM = 'UNAUTHENTICATED'
 
 
 class SessionKeyError(Exception): pass
@@ -43,7 +44,8 @@ def register_auth_module(auth_module):
   _AUTH_MODULES[auth_module.mechanism] = auth_module
 
 
-def make_session_key(auth_mechanism='UNAUTHENTICATED'):
+# TODO(maxim): drop in AURORA-1229
+def make_session_key(auth_mechanism=DEFAULT_AUTH_MECHANISM):
   """
     Attempts to create a session key by calling the auth module registered to the auth mechanism.
     If an auth module does not exist for an auth mechanism, an InsecureAuthModule will be used.
@@ -58,3 +60,13 @@ def make_session_key(auth_mechanism='UNAUTHENTICATED'):
     raise SessionKeyError('Expected %r but got %r from auth module %r' % (
       SessionKey, session_key.__class__, auth_module))
   return session_key
+
+
+def get_auth_handler(auth_mechanism=DEFAULT_AUTH_MECHANISM):
+  """Returns an auth handler to be used in Thrift transport layer."""
+  if not _AUTH_MODULES:
+    raise SessionKeyError('No auth modules have been registered. Please call register_auth_module.')
+
+  auth_module = _AUTH_MODULES.get(auth_mechanism) or _INSECURE_AUTH_MODULE
+  log.debug('Using auth module: %r' % auth_module)
+  return auth_module.auth()

http://git-wip-us.apache.org/repos/asf/aurora/blob/eb9e4088/src/main/python/apache/aurora/common/auth/kerberos.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/common/auth/kerberos.py b/src/main/python/apache/aurora/common/auth/kerberos.py
new file mode 100644
index 0000000..2d782b6
--- /dev/null
+++ b/src/main/python/apache/aurora/common/auth/kerberos.py
@@ -0,0 +1,39 @@
+#
+# 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 requests_kerberos import DISABLED, HTTPKerberosAuth
+
+from .auth_module import AuthModule
+
+
+class KerberosAuthModule(AuthModule):
+  @property
+  def mechanism(self):
+    return 'KERBEROS'
+
+  def payload(self):
+    """NOTE: until AURORA-1229 is addressed, using "Kerberized" client in production in a backwards
+             compatible way will require a new custom module that will override this method to
+             return the currently used payload (security blob used in SessionKey).
+    """
+    return ''
+
+  def auth(self):
+    # While SPNEGO supports mutual authentication of the response, it does not assert the validity
+    # of the response payload, only the identity of the server. Thus the scheduler will not set
+    # the WWW-Authenticate response header and the client will disable mutual authentication.
+    # In order to achieve communication with the scheduler subject to confidentiality and integrity
+    # constraints the client must connect to the scheduler API via HTTPS. Kerberos is thus only
+    # used to authenticate the client to the server.
+    return HTTPKerberosAuth(mutual_authentication=DISABLED)

http://git-wip-us.apache.org/repos/asf/aurora/blob/eb9e4088/src/main/python/apache/aurora/common/transport.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/common/transport.py b/src/main/python/apache/aurora/common/transport.py
index 395f8a9..eefe8f7 100644
--- a/src/main/python/apache/aurora/common/transport.py
+++ b/src/main/python/apache/aurora/common/transport.py
@@ -54,9 +54,12 @@ class TRequestsTransport(TTransportBase):
 
     :param uri: The endpoint uri
     :type uri: str
-    :keyword auth: The requests authentication context.
-    :keyword session_factory: A callable that returns a requests session.
-    :keyword user_agent: The value to use for the User-Agent header.
+    :keyword auth: The requests authentication context
+    :type auth: requests.auth.AuthBase
+    :keyword session_factory: A callable that returns a requests session
+    :type session_factory: requests.Session
+    :keyword user_agent: The value to use for the User-Agent header
+    :type user_agent: str
     """
     self._session = None
     self.__session_factory = session_factory
@@ -71,6 +74,9 @@ class TRequestsTransport(TTransportBase):
     except ValueError:
       raise TTransportException('Failed to parse uri %r' % (uri,))
     self.__timeout = None
+    if auth is not None and not isinstance(auth, requests.auth.AuthBase):
+      raise TypeError('Invalid auth type. Expected: %s but got %s'
+                      % (requests.auth.AuthBase.__name__, auth.__class__.__name__))
     self.__auth = auth
 
     # Silence requests logs so we don't get messages for every HTTP connection.

http://git-wip-us.apache.org/repos/asf/aurora/blob/eb9e4088/src/test/python/apache/aurora/client/api/test_scheduler_client.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/api/test_scheduler_client.py b/src/test/python/apache/aurora/client/api/test_scheduler_client.py
index 0a61948..9319728 100644
--- a/src/test/python/apache/aurora/client/api/test_scheduler_client.py
+++ b/src/test/python/apache/aurora/client/api/test_scheduler_client.py
@@ -19,6 +19,7 @@ import unittest
 import mock
 import pytest
 from mox import IgnoreArg, IsA, Mox
+from requests.auth import AuthBase
 from thrift.transport import TTransport
 from twitter.common.quantity import Amount, Time
 from twitter.common.zookeeper.kazoo_client import TwitterKazooClient
@@ -329,6 +330,10 @@ class TestSchedulerProxyAdminInjection(TestSchedulerProxyInjection):
     self.make_scheduler_proxy().rewriteConfigs(RewriteConfigsRequest())
 
 
+def mock_auth():
+  return mock.create_autospec(spec=AuthBase, instance=True)
+
+
 @pytest.mark.parametrize('scheme', ('http', 'https'))
 def test_url_when_not_connected_and_cluster_has_no_proxy_url(scheme):
   host = 'some-host.example.com'
@@ -356,6 +361,7 @@ def test_url_when_not_connected_and_cluster_has_no_proxy_url(scheme):
   def make_mock_client(proxy_url):
     client = scheduler_client.ZookeeperSchedulerClient(
         Cluster(proxy_url=proxy_url),
+        auth=None,
         user_agent='Some-User-Agent',
         _deadline=lambda x, **kws: x())
     client.get_scheduler_serverset = mock.MagicMock(return_value=(mock_zk, service_endpoints))
@@ -387,7 +393,7 @@ def test_connect_scheduler(mock_client):
   mock_client.return_value.open.side_effect = [TTransport.TTransportException, True]
   mock_time = mock.create_autospec(spec=time, instance=True)
 
-  client = scheduler_client.SchedulerClient('Some-User-Agent', verbose=True)
+  client = scheduler_client.SchedulerClient(mock_auth(), 'Some-User-Agent', verbose=True)
   client._connect_scheduler('https://scheduler.example.com:1337', mock_time)
 
   assert mock_client.return_value.open.has_calls(mock.call(), mock.call())
@@ -400,14 +406,15 @@ def test_connect_scheduler_with_user_agent(mock_transport):
   mock_transport.return_value.open.side_effect = [TTransport.TTransportException, True]
   mock_time = mock.create_autospec(spec=time, instance=True)
 
+  auth = mock_auth()
   user_agent = 'Some-User-Agent'
 
-  client = scheduler_client.SchedulerClient(user_agent, verbose=True)
+  client = scheduler_client.SchedulerClient(auth, user_agent, verbose=True)
 
   uri = 'https://scheduler.example.com:1337'
   client._connect_scheduler(uri, mock_time)
 
-  mock_transport.assert_called_once_with(uri, user_agent=user_agent)
+  mock_transport.assert_called_once_with(uri, auth=auth, user_agent=user_agent)
 
 
 @mock.patch('apache.aurora.client.api.scheduler_client.SchedulerClient',
@@ -445,7 +452,58 @@ def test_connect_direct_scheduler_with_user_agent(mock_transport):
   user_agent = 'Some-User-Agent'
   uri = 'https://scheduler.example.com:1337'
 
-  client = scheduler_client.DirectSchedulerClient(uri, verbose=True, user_agent=user_agent)
+  client = scheduler_client.DirectSchedulerClient(
+      uri,
+      auth=None,
+      verbose=True,
+      user_agent=user_agent)
+
+  client._connect_scheduler(uri, mock_time)
+
+  mock_transport.assert_called_once_with(uri, auth=None, user_agent=user_agent)
+
+
+@mock.patch('apache.aurora.client.api.scheduler_client.TRequestsTransport', spec=TRequestsTransport)
+def test_connect_zookeeper_client_with_auth(mock_transport):
+  mock_transport.return_value.open.side_effect = [TTransport.TTransportException, True]
+  mock_time = mock.create_autospec(spec=time, instance=True)
+
+  user_agent = 'Some-User-Agent'
+  uri = 'https://scheduler.example.com:1337'
+  auth = mock_auth()
+  cluster = Cluster(zk='zk', zk_port='2181')
+
+  def auth_factory(_):
+    return auth
+
+  client = scheduler_client.SchedulerClient.get(
+      cluster,
+      auth_factory=auth_factory,
+      user_agent=user_agent)
+
+  client._connect_scheduler(uri, mock_time)
+
+  mock_transport.assert_called_once_with(uri, auth=auth, user_agent=user_agent)
+
+
+@mock.patch('apache.aurora.client.api.scheduler_client.TRequestsTransport', spec=TRequestsTransport)
+def test_connect_direct_client_with_auth(mock_transport):
+  mock_transport.return_value.open.side_effect = [TTransport.TTransportException, True]
+  mock_time = mock.create_autospec(spec=time, instance=True)
+
+  user_agent = 'Some-User-Agent'
+  uri = 'https://scheduler.example.com:1337'
+  auth = mock_auth()
+  cluster = Cluster(scheduler_uri='uri')
+
+  def auth_factory(_):
+    return auth
+
+  client = scheduler_client.SchedulerClient.get(
+      cluster,
+      auth_factory=auth_factory,
+      user_agent=user_agent)
+
   client._connect_scheduler(uri, mock_time)
 
-  mock_transport.assert_called_once_with(uri, user_agent=user_agent)
+  mock_transport.assert_called_once_with(uri, auth=auth, user_agent=user_agent)

http://git-wip-us.apache.org/repos/asf/aurora/blob/eb9e4088/src/test/python/apache/aurora/common/test_transport.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/common/test_transport.py b/src/test/python/apache/aurora/common/test_transport.py
index b78e0b3..e4f93ef 100644
--- a/src/test/python/apache/aurora/common/test_transport.py
+++ b/src/test/python/apache/aurora/common/test_transport.py
@@ -17,7 +17,7 @@ from threading import Thread
 
 import pytest
 import requests
-from mock import create_autospec, Mock
+from mock import ANY, call, create_autospec, Mock
 from requests import exceptions as request_exceptions
 from thrift.protocol import TJSONProtocol
 from thrift.server import THttpServer
@@ -135,3 +135,31 @@ def test_transport_applies_default_user_agent_if_no_factory_provided():
   transport = TRequestsTransport('http://localhost:12345')
   transport.open()
   assert transport._session.headers['User-Agent'] == DEFAULT_USER_AGENT
+
+
+def test_auth_type_valid():
+  response = create_autospec(spec=requests.Response, instance=True)
+  response.headers = {'header1': 'data', 'header2': 'data2'}
+  response.raise_for_status.side_effect = requests.exceptions.HTTPError()
+
+  session = create_autospec(spec=requests.Session, instance=True)
+  session.headers = {}
+  session.post.return_value = response
+
+  auth = requests.auth.AuthBase()
+  transport = TRequestsTransport('http://localhost:1', auth=auth, session_factory=lambda: session)
+  protocol = TJSONProtocol.TJSONProtocol(transport)
+  client = ReadOnlyScheduler.Client(protocol)
+
+  with pytest.raises(TTransport.TTransportException):
+    client.getRoleSummary()
+
+  transport.close()
+
+  session.post.mock_calls = (call(ANY, data=ANY, timeout=ANY, auth=auth))
+
+
+def test_auth_type_invalid():
+  with pytest.raises(TypeError) as e:
+    TRequestsTransport('http://localhost:1', auth="auth")
+  assert e.value.message == 'Invalid auth type. Expected: AuthBase but got str'

http://git-wip-us.apache.org/repos/asf/aurora/blob/eb9e4088/src/test/sh/org/apache/aurora/e2e/test_kerberos_end_to_end.sh
----------------------------------------------------------------------
diff --git a/src/test/sh/org/apache/aurora/e2e/test_kerberos_end_to_end.sh b/src/test/sh/org/apache/aurora/e2e/test_kerberos_end_to_end.sh
index 47d22ee..6e9e3b2 100755
--- a/src/test/sh/org/apache/aurora/e2e/test_kerberos_end_to_end.sh
+++ b/src/test/sh/org/apache/aurora/e2e/test_kerberos_end_to_end.sh
@@ -1,4 +1,19 @@
 #!/bin/bash
+#
+# 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.
+#
+#
+# An integration test for the client, using the vagrant environment as a testbed.
 set -eux
 
 readonly KRB5_MAJOR_MINOR=1.13
@@ -29,7 +44,7 @@ function enter_testrealm {
 }
 
 function await_scheduler_ready {
-  while ! curl -s localhost:8081/vars | grep framework_registered; do
+  while ! curl -s localhost:8081/vars | grep "framework_registered 1"; do
     sleep 3
   done
 }
@@ -41,19 +56,27 @@ function snapshot_as {
   kinit -k -t "testdir/${principal}.keytab" $principal
   curl -u : --negotiate -w '%{http_code}\n' \
     -o $(printf $SNAPSHOT_RESPONSE_OUTFILE $principal) \
-    -s 'http://localhost:8081/api' \
+    -s 'http://192.168.33.7:8081/api' \
     --data-binary "$SNAPSHOT_RPC_DATA"
   kdestroy
 }
 
-function test_snapshot {
+function setup {
   cat >> $KRB5_CONFIG <<EOF
 [domain_realm]
   .local = KRBTEST.COM
 EOF
-  kadmin.local -q "addprinc -randkey HTTP/localhost"
-  rm -f testdir/HTTP-localhost.keytab
-  kadmin.local -q "ktadd -keytab testdir/HTTP-localhost.keytab HTTP/localhost"
+
+  aurorabuild all
+  sudo cp /vagrant/examples/vagrant/upstart/aurora-scheduler-kerberos.conf \
+    /etc/init/aurora-scheduler-kerberos.conf
+  sudo stop aurora-scheduler || true
+  sudo start aurora-scheduler-kerberos
+  await_scheduler_ready
+
+  kadmin.local -q "addprinc -randkey HTTP/192.168.33.7"
+  rm -f testdir/HTTP-192.168.33.7.keytab.keytab
+  kadmin.local -q "ktadd -keytab testdir/HTTP-192.168.33.7.keytab HTTP/192.168.33.7"
 
   kadmin.local -q "addprinc -randkey vagrant"
   rm -f testdir/vagrant.keytab
@@ -66,13 +89,9 @@ EOF
   kadmin.local -q "addprinc -randkey root"
   rm -f testdir/root.keytab
   kadmin.local -q "ktadd -keytab testdir/root.keytab root"
+}
 
-  sudo cp /vagrant/examples/vagrant/upstart/aurora-scheduler-kerberos.conf \
-    /etc/init/aurora-scheduler-kerberos.conf
-  aurorabuild scheduler
-  sudo stop aurora-scheduler || true
-  sudo start aurora-scheduler-kerberos
-  await_scheduler_ready
+function test_snapshot {
   snapshot_as vagrant
   cat snapshot-response.vagrant.json
   grep -q 'lacks permission' snapshot-response.vagrant.json
@@ -84,7 +103,17 @@ EOF
   grep -qv 'lacks permission' snapshot-response.root.json
 }
 
+function test_clients {
+  sudo cp /vagrant/examples/vagrant/clusters_kerberos.json /etc/aurora/clusters.json
+
+  kinit -k -t "testdir/root.keytab" root
+  aurora_admin set_quota devcluster kerberos-test 0.0 0MB 0MB /dev/null 2>&1 | grep 'OK' | true
+  aurora update pause devcluster/role/env/job /dev/null 2>&1 | grep 'No active update found' | true
+  kdestroy
+}
+
 function tear_down {
+  sudo cp /vagrant/examples/vagrant/clusters.json /etc/aurora/clusters.json
   sudo stop aurora-scheduler-kerberos || true
   sudo rm -f /etc/init/aurora-scheduler-kerberos.conf
   sudo start aurora-scheduler || true
@@ -97,7 +126,9 @@ function main {
     enter_testrealm "$@"
   else
     trap tear_down EXIT
+    setup
     test_snapshot
+    test_clients
     set +x
     echo
     echo '*** OK (All tests passed) ***'