You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by ke...@apache.org on 2015/10/23 00:47:35 UTC

aurora git commit: Always set SessionKey to empty in the client.

Repository: aurora
Updated Branches:
  refs/heads/master 4eeec7a75 -> 2f173c653


Always set SessionKey to empty in the client.

Always set SessionKey to empty on the client, as it's now ignored by
the scheduler.

Testing Done:
./pants test.pytest --no-fast src/test/python/::

./src/test/sh/org/apache/aurora/e2e/test_end_to_end.sh

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


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

Branch: refs/heads/master
Commit: 2f173c65337445ad765395f596ea346dc87544fa
Parents: 4eeec7a
Author: Kevin Sweeney <ke...@apache.org>
Authored: Thu Oct 22 15:47:09 2015 -0700
Committer: Kevin Sweeney <ke...@apache.org>
Committed: Thu Oct 22 15:47:09 2015 -0700

----------------------------------------------------------------------
 .../thrift/org/apache/aurora/gen/api.thrift     |  8 +-------
 .../python/apache/aurora/client/api/__init__.py |  5 +----
 .../aurora/client/api/scheduler_client.py       | 20 ++++---------------
 .../apache/aurora/common/auth/auth_module.py    | 17 ----------------
 .../aurora/common/auth/auth_module_manager.py   | 21 --------------------
 .../apache/aurora/client/api/test_restarter.py  |  3 ++-
 .../aurora/client/api/test_scheduler_client.py  |  2 +-
 .../apache/aurora/client/api/test_updater.py    |  3 ++-
 8 files changed, 11 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/2f173c65/api/src/main/thrift/org/apache/aurora/gen/api.thrift
----------------------------------------------------------------------
diff --git a/api/src/main/thrift/org/apache/aurora/gen/api.thrift b/api/src/main/thrift/org/apache/aurora/gen/api.thrift
index f0d4ef8..b262884 100644
--- a/api/src/main/thrift/org/apache/aurora/gen/api.thrift
+++ b/api/src/main/thrift/org/apache/aurora/gen/api.thrift
@@ -51,14 +51,8 @@ struct Identity {
   2: string user
 }
 
+/** Deprecated, to be removed in 0.11.0 (https://issues.apache.org/jira/browse/AURORA-1229). */
 struct SessionKey {
-  /**
-   * The name of the authentication mechanism, which instructs the server how to interpret the data
-   * field.
-   */
-  4: optional string mechanism
-  /** A blob of data that the server may use for authentication. */
-  5: optional binary data
 }
 
 struct ResourceAggregate {

http://git-wip-us.apache.org/repos/asf/aurora/blob/2f173c65/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 5847ca8..6f07a30 100644
--- a/src/main/python/apache/aurora/client/api/__init__.py
+++ b/src/main/python/apache/aurora/client/api/__init__.py
@@ -18,7 +18,6 @@ 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.auth_module_manager import make_session_key
 from apache.aurora.common.cluster import Cluster
 
 from .restarter import Restarter
@@ -52,8 +51,7 @@ class AuroraClientAPI(object):
       self,
       cluster,
       user_agent,
-      verbose=False,
-      session_key_factory=make_session_key):
+      verbose=False):
 
     if not isinstance(cluster, Cluster):
       raise TypeError('AuroraClientAPI expects instance of Cluster for "cluster", got %s' %
@@ -61,7 +59,6 @@ class AuroraClientAPI(object):
     self._scheduler_proxy = SchedulerProxy(
         cluster,
         verbose=verbose,
-        session_key_factory=session_key_factory,
         user_agent=user_agent)
     self._cluster = cluster
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/2f173c65/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 8e91788..7aa0619 100644
--- a/src/main/python/apache/aurora/client/api/scheduler_client.py
+++ b/src/main/python/apache/aurora/client/api/scheduler_client.py
@@ -26,17 +26,13 @@ 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.auth_module_manager import (
-    SessionKeyError,
-    get_auth_handler,
-    make_session_key
-)
+from apache.aurora.common.auth.auth_module_manager import get_auth_handler
 from apache.aurora.common.cluster import Cluster
 from apache.aurora.common.transport import TRequestsTransport
 
 from gen.apache.aurora.api import AuroraAdmin, ReadOnlyScheduler
 from gen.apache.aurora.api.constants import THRIFT_API_VERSION
-from gen.apache.aurora.api.ttypes import ResponseCode
+from gen.apache.aurora.api.ttypes import ResponseCode, SessionKey
 
 try:
   from urlparse import urljoin
@@ -219,12 +215,10 @@ class SchedulerProxy(object):
   class APIVersionError(Error): pass
   class ThriftInternalError(Error): pass
 
-  def __init__(self, cluster, verbose=False, session_key_factory=make_session_key, **kwargs):
-    """A callable session_key_factory should be provided for authentication"""
+  def __init__(self, cluster, verbose=False, **kwargs):
     self.cluster = cluster
     # TODO(Sathya): Make this a part of cluster trait when authentication is pushed to the transport
     # layer.
-    self._session_key_factory = session_key_factory
     self._client = self._scheduler_client = None
     self.verbose = verbose
     self._lock = threading.RLock()
@@ -255,12 +249,6 @@ class SchedulerProxy(object):
   def scheduler_client(self):
     return self._scheduler_client
 
-  def session_key(self):
-    try:
-      return self._session_key_factory(self.cluster.auth_mechanism)
-    except SessionKeyError as e:
-      raise self.AuthError('Unable to create session key %s' % e)
-
   def _construct_scheduler(self):
     """
       Populates:
@@ -300,7 +288,7 @@ class SchedulerProxy(object):
             time.time() - start) < self.RPC_MAXIMUM_WAIT.as_(Time.SECONDS):
 
           # Only automatically append a SessionKey if this is not part of the read-only API.
-          auth_args = () if hasattr(ReadOnlyScheduler.Iface, method_name) else (self.session_key(),)
+          auth_args = () if hasattr(ReadOnlyScheduler.Iface, method_name) else (SessionKey(),)
           try:
             method = getattr(self.client(), method_name)
             if not callable(method):

http://git-wip-us.apache.org/repos/asf/aurora/blob/2f173c65/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 53a3182..e655cad 100644
--- a/src/main/python/apache/aurora/common/auth/auth_module.py
+++ b/src/main/python/apache/aurora/common/auth/auth_module.py
@@ -11,13 +11,10 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-
 from abc import abstractmethod, abstractproperty
 
 from twitter.common.lang import Interface
 
-from gen.apache.aurora.api.ttypes import SessionKey
-
 
 class AuthModule(Interface):
   @abstractproperty
@@ -26,17 +23,6 @@ class AuthModule(Interface):
     ":rtype: string
     """
 
-  # TODO(maxim): drop this in AURORA-1229
-  @abstractmethod
-  def payload(self):
-    """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.
@@ -49,8 +35,5 @@ class InsecureAuthModule(AuthModule):
   def mechanism(self):
     return 'UNAUTHENTICATED'
 
-  def payload(self):
-    return 'UNAUTHENTICATED'
-
   def auth(self):
     return None

http://git-wip-us.apache.org/repos/asf/aurora/blob/2f173c65/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 73c6bd7..07a2730 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
@@ -11,13 +11,10 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-
 from twitter.common import log
 
 from .auth_module import AuthModule, InsecureAuthModule
 
-from gen.apache.aurora.api.ttypes import SessionKey
-
 _INSECURE_AUTH_MODULE = InsecureAuthModule()
 _AUTH_MODULES = {
   _INSECURE_AUTH_MODULE.mechanism: _INSECURE_AUTH_MODULE,
@@ -44,24 +41,6 @@ def register_auth_module(auth_module):
   _AUTH_MODULES[auth_module.mechanism] = auth_module
 
 
-# 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.
-  """
-  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)
-  session_key = auth_module()
-  if not isinstance(session_key, SessionKey):
-    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:

http://git-wip-us.apache.org/repos/asf/aurora/blob/2f173c65/src/test/python/apache/aurora/client/api/test_restarter.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/api/test_restarter.py b/src/test/python/apache/aurora/client/api/test_restarter.py
index 3b14d88..f31d60d 100644
--- a/src/test/python/apache/aurora/client/api/test_restarter.py
+++ b/src/test/python/apache/aurora/client/api/test_restarter.py
@@ -34,10 +34,11 @@ from gen.apache.aurora.api.ttypes import (
     ScheduleStatus,
     ScheduleStatusResult,
     ServerInfo,
+    SessionKey,
     TaskConfig
 )
 
-SESSION_KEY = 'test_session'
+SESSION_KEY = SessionKey()
 CLUSTER = 'east'
 JOB = AuroraJobKey(CLUSTER, 'johndoe', 'test', 'test_job')
 HEALTH_CHECK_INTERVAL_SECONDS = 5

http://git-wip-us.apache.org/repos/asf/aurora/blob/2f173c65/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 10e8ebb..60d222f 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
@@ -67,7 +67,7 @@ def test_coverage():
           'No test defined for RPC %s' % rpc_name)
 
 
-SESSION = SessionKey(mechanism='test', data='test')
+SESSION = SessionKey()
 
 
 class TestSchedulerProxy(scheduler_client.SchedulerProxy):

http://git-wip-us.apache.org/repos/asf/aurora/blob/2f173c65/src/test/python/apache/aurora/client/api/test_updater.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/api/test_updater.py b/src/test/python/apache/aurora/client/api/test_updater.py
index 8793555..b7b1653 100644
--- a/src/test/python/apache/aurora/client/api/test_updater.py
+++ b/src/test/python/apache/aurora/client/api/test_updater.py
@@ -53,6 +53,7 @@ from gen.apache.aurora.api.ttypes import (
     ScheduledTask,
     ScheduleStatusResult,
     ServerInfo,
+    SessionKey,
     TaskConfig,
     TaskConstraint,
     TaskQuery,
@@ -148,7 +149,7 @@ class UpdaterTest(TestCase):
     self._name = 'jimbob'
     self._env = 'test'
     self._job_key = JobKey(name=self._name, environment=self._env, role=self._role)
-    self._session_key = 'test_session'
+    self._session_key = SessionKey()
     self._lock = 'test_lock'
     self._instance_watcher = MockObject(InstanceWatcher)
     self._job_monitor = MockObject(JobMonitor)