You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by wi...@apache.org on 2014/12/19 22:59:34 UTC

incubator-aurora git commit: Add custom user agent for Aurora v1, Aurora v2 and Aurora Admin clients.

Repository: incubator-aurora
Updated Branches:
  refs/heads/master 393b2a30a -> 8c841060b


Add custom user agent for Aurora v1, Aurora v2 and Aurora Admin clients.

Testing Done:
./pants build src/test/python/apache/aurora::

Also ran e2e tests for v1 and v2 and verified the user agent in the
scheduler logs.

Bugs closed: AURORA-934

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


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

Branch: refs/heads/master
Commit: 8c841060bb4457518a372f0f84053b2dec0e7034
Parents: 393b2a3
Author: Joshua Cohen <jc...@twopensource.com>
Authored: Fri Dec 19 13:59:26 2014 -0800
Committer: Brian Wickman <wi...@apache.org>
Committed: Fri Dec 19 13:59:26 2014 -0800

----------------------------------------------------------------------
 src/main/python/apache/aurora/client/BUILD      |  2 +
 .../python/apache/aurora/client/api/__init__.py | 13 +++-
 .../apache/aurora/client/api/command_runner.py  |  6 +-
 .../aurora/client/api/scheduler_client.py       | 27 +++----
 src/main/python/apache/aurora/client/base.py    | 20 +++++
 .../python/apache/aurora/client/cli/context.py  |  4 +-
 .../python/apache/aurora/client/commands/BUILD  |  2 +
 .../apache/aurora/client/commands/admin.py      | 63 ++++++---------
 .../apache/aurora/client/commands/core.py       | 36 +++++----
 .../python/apache/aurora/client/commands/ssh.py |  4 +-
 src/main/python/apache/aurora/client/factory.py |  8 +-
 .../python/apache/aurora/common/transport.py    | 36 ++++++---
 .../aurora/client/api/test_scheduler_client.py  | 28 +++++--
 .../apache/aurora/client/cli/test_diff.py       |  2 +
 .../apache/aurora/client/commands/test_admin.py | 80 +++++++++++---------
 .../aurora/client/commands/test_admin_sla.py    | 18 ++---
 .../aurora/client/commands/test_create.py       | 12 +--
 .../apache/aurora/client/commands/test_diff.py  |  2 +
 .../apache/aurora/common/test_transport.py      | 17 ++++-
 19 files changed, 236 insertions(+), 144 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8c841060/src/main/python/apache/aurora/client/BUILD
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/BUILD b/src/main/python/apache/aurora/client/BUILD
index 2a66825..a2a29ff 100644
--- a/src/main/python/apache/aurora/client/BUILD
+++ b/src/main/python/apache/aurora/client/BUILD
@@ -29,6 +29,7 @@ python_library(
     '3rdparty/python:twitter.common.app',
     '3rdparty/python:twitter.common.log',
     'api/src/main/thrift/org/apache/aurora/gen:py-thrift',
+    'src/main/python/apache/aurora/common',
   ]
 )
 
@@ -61,6 +62,7 @@ python_library(
     ':base',
     '3rdparty/python:twitter.common.app',
     'src/main/python/apache/aurora/client/hooks',
+    'src/main/python/apache/aurora/client/hooks',
     'src/main/python/apache/aurora/common:cluster',
     'src/main/python/apache/aurora/common:clusters',
   ]

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8c841060/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 9977b41..64f0804 100644
--- a/src/main/python/apache/aurora/client/api/__init__.py
+++ b/src/main/python/apache/aurora/client/api/__init__.py
@@ -48,12 +48,21 @@ class AuroraClientAPI(object):
   class ThriftInternalError(Error): pass
   class UpdateConfigError(Error): pass
 
-  def __init__(self, cluster, verbose=False, session_key_factory=make_session_key):
+  def __init__(
+      self,
+      cluster,
+      user_agent,
+      verbose=False,
+      session_key_factory=make_session_key):
+
     if not isinstance(cluster, Cluster):
       raise TypeError('AuroraClientAPI expects instance of Cluster for "cluster", got %s' %
           type(cluster))
     self._scheduler_proxy = SchedulerProxy(
-        cluster, verbose=verbose, session_key_factory=session_key_factory)
+        cluster,
+        verbose=verbose,
+        session_key_factory=session_key_factory,
+        user_agent=user_agent)
     self._cluster = cluster
 
   @property

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8c841060/src/main/python/apache/aurora/client/api/command_runner.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/api/command_runner.py b/src/main/python/apache/aurora/client/api/command_runner.py
index 4d8f010..48cb567 100644
--- a/src/main/python/apache/aurora/client/api/command_runner.py
+++ b/src/main/python/apache/aurora/client/api/command_runner.py
@@ -23,7 +23,7 @@ from pystachio import Environment, Required, String
 from twitter.common import log
 
 from apache.aurora.client.api import AuroraClientAPI
-from apache.aurora.client.base import combine_messages
+from apache.aurora.client.base import AURORA_V1_USER_AGENT_NAME, combine_messages
 from apache.aurora.common.cluster import Cluster
 from apache.aurora.config.schema.base import MesosContext
 from apache.thermos.config.schema import ThermosContext
@@ -101,7 +101,9 @@ class DistributedCommandRunner(object):
   def __init__(self, cluster, role, env, jobs, ssh_user=None,
       log_fn=log.log):
     self._cluster = cluster
-    self._api = AuroraClientAPI(cluster=cluster)
+    self._api = AuroraClientAPI(
+        cluster=cluster,
+        user_agent=AURORA_V1_USER_AGENT_NAME)
     self._role = role
     self._env = env
     self._jobs = jobs

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8c841060/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 4ae18f0..a319a1e 100644
--- a/src/main/python/apache/aurora/client/api/scheduler_client.py
+++ b/src/main/python/apache/aurora/client/api/scheduler_client.py
@@ -67,12 +67,13 @@ class SchedulerClient(object):
     if cluster.zk:
       return ZookeeperSchedulerClient(cluster, port=cluster.zk_port, **kwargs)
     elif cluster.scheduler_uri:
-      return DirectSchedulerClient(cluster.scheduler_uri)
+      return DirectSchedulerClient(cluster.scheduler_uri, **kwargs)
     else:
       raise ValueError('"cluster" does not specify zk or scheduler_uri')
 
-  def __init__(self, verbose=False):
+  def __init__(self, user_agent, verbose=False):
     self._client = None
+    self._user_agent = user_agent
     self._verbose = verbose
 
   def get_thrift_client(self):
@@ -85,23 +86,22 @@ class SchedulerClient(object):
   def _connect(self):
     return None
 
-  @classmethod
-  def _connect_scheduler(cls, uri, clock=time):
-    transport = TRequestsTransport(uri)
+  def _connect_scheduler(self, uri, clock=time):
+    transport = TRequestsTransport(uri, user_agent=self._user_agent)
     protocol = TJSONProtocol.TJSONProtocol(transport)
     schedulerClient = AuroraAdmin.Client(protocol)
-    for _ in range(cls.THRIFT_RETRIES):
+    for _ in range(self.THRIFT_RETRIES):
       try:
         transport.open()
         return schedulerClient
       except TTransport.TTransportException:
-        clock.sleep(cls.RETRY_TIMEOUT.as_(Time.SECONDS))
+        clock.sleep(self.RETRY_TIMEOUT.as_(Time.SECONDS))
         continue
       except Exception as e:
         # Monkey-patched proxies, like socks, can generate a proxy error here.
         # without adding a dependency, we can't catch those in a more specific way.
-        raise cls.CouldNotConnect('Connection to scheduler failed: %s' % e)
-    raise cls.CouldNotConnect('Could not connect to %s' % uri)
+        raise self.CouldNotConnect('Connection to scheduler failed: %s' % e)
+    raise self.CouldNotConnect('Could not connect to %s' % uri)
 
 
 class ZookeeperSchedulerClient(SchedulerClient):
@@ -117,8 +117,8 @@ class ZookeeperSchedulerClient(SchedulerClient):
     zk = TwitterKazooClient.make(str('%s:%s' % (cluster.zk, port)), verbose=verbose)
     return zk, ServerSet(zk, cluster.scheduler_zk_path, **kw)
 
-  def __init__(self, cluster, port=2181, verbose=False, _deadline=deadline):
-    SchedulerClient.__init__(self, verbose=verbose)
+  def __init__(self, cluster, port=2181, verbose=False, _deadline=deadline, **kwargs):
+    SchedulerClient.__init__(self, verbose=verbose, **kwargs)
     self._cluster = cluster
     self._zkport = port
     self._endpoint = None
@@ -212,7 +212,7 @@ class SchedulerProxy(object):
   class APIVersionError(Error): pass
   class ThriftInternalError(Error): pass
 
-  def __init__(self, cluster, verbose=False, session_key_factory=make_session_key):
+  def __init__(self, cluster, verbose=False, session_key_factory=make_session_key, **kwargs):
     """A callable session_key_factory should be provided for authentication"""
     self.cluster = cluster
     # TODO(Sathya): Make this a part of cluster trait when authentication is pushed to the transport
@@ -222,6 +222,7 @@ class SchedulerProxy(object):
     self.verbose = verbose
     self._lock = threading.RLock()
     self._terminating = threading.Event()
+    self._kwargs = kwargs
 
   def with_scheduler(method):
     """Decorator magic to make sure a connection is made to the scheduler"""
@@ -259,7 +260,7 @@ class SchedulerProxy(object):
         self._scheduler_client
         self._client
     """
-    self._scheduler_client = SchedulerClient.get(self.cluster, verbose=self.verbose)
+    self._scheduler_client = SchedulerClient.get(self.cluster, verbose=self.verbose, **self._kwargs)
     assert self._scheduler_client, "Could not find scheduler (cluster = %s)" % self.cluster.name
     start = time.time()
     while (time.time() - start) < self.CONNECT_MAXIMUM_WAIT.as_(Time.SECONDS):

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8c841060/src/main/python/apache/aurora/client/base.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/base.py b/src/main/python/apache/aurora/client/base.py
index a897e87..258c1fa 100644
--- a/src/main/python/apache/aurora/client/base.py
+++ b/src/main/python/apache/aurora/client/base.py
@@ -20,6 +20,8 @@ from urlparse import urljoin
 
 from twitter.common import app, log
 
+from apache.aurora.common.pex_version import pex_version, UnknownVersion
+
 from gen.apache.aurora.api.ttypes import ResponseCode
 
 
@@ -239,3 +241,21 @@ Available commands:
 For more help on an individual command:
     %s help <command>
 """ % (usage, app.name())
+
+
+AURORA_V1_USER_AGENT_NAME = 'Aurora v1'
+AURORA_V2_USER_AGENT_NAME = 'Aurora v2'
+AURORA_ADMIN_USER_AGENT_NAME = 'Aurora Admin'
+
+UNKNOWN_CLIENT_VERSION = 'Unknown Version'
+
+
+def user_agent(agent_name='Aurora'):
+  """Generate a user agent containing the specified agent name and the details of the current
+     client version."""
+  try:
+    build_info = '%s-%s' % pex_version(sys.argv[0])
+  except UnknownVersion:
+    build_info = UNKNOWN_CLIENT_VERSION
+
+  return '%s;%s' % (agent_name, build_info)

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8c841060/src/main/python/apache/aurora/client/cli/context.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/context.py b/src/main/python/apache/aurora/client/cli/context.py
index b1ffc05..f062afc 100644
--- a/src/main/python/apache/aurora/client/cli/context.py
+++ b/src/main/python/apache/aurora/client/cli/context.py
@@ -21,7 +21,7 @@ from fnmatch import fnmatch
 
 from pystachio import Ref
 
-from apache.aurora.client.base import combine_messages, synthesize_url
+from apache.aurora.client.base import AURORA_V2_USER_AGENT_NAME, combine_messages, synthesize_url
 from apache.aurora.client.cli import (
     Context,
     EXIT_API_ERROR,
@@ -79,7 +79,7 @@ class AuroraCommandContext(Context):
     session.
     """
     if cluster not in self.apis:
-      api = make_client(cluster)
+      api = make_client(cluster, AURORA_V2_USER_AGENT_NAME)
       self.apis[cluster] = api
     return self.apis[cluster]
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8c841060/src/main/python/apache/aurora/client/commands/BUILD
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/commands/BUILD b/src/main/python/apache/aurora/client/commands/BUILD
index f256483..78a2f57 100644
--- a/src/main/python/apache/aurora/client/commands/BUILD
+++ b/src/main/python/apache/aurora/client/commands/BUILD
@@ -32,6 +32,8 @@ python_library(
     'src/main/python/apache/aurora/admin:util',
     'src/main/python/apache/aurora/client/api',
     'src/main/python/apache/aurora/client:base',
+    'src/main/python/apache/aurora/client:config',
+    'src/main/python/apache/aurora/client:factory',
     'src/main/python/apache/aurora/common:clusters',
     'api/src/main/thrift/org/apache/aurora/gen:py-thrift',
   ]

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8c841060/src/main/python/apache/aurora/client/commands/admin.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/commands/admin.py b/src/main/python/apache/aurora/client/commands/admin.py
index 4b86c5a..b7cbba0 100644
--- a/src/main/python/apache/aurora/client/commands/admin.py
+++ b/src/main/python/apache/aurora/client/commands/admin.py
@@ -32,9 +32,9 @@ from apache.aurora.admin.admin_util import (
     parse_sla_percentage,
     print_results
 )
-from apache.aurora.client.api import AuroraClientAPI
 from apache.aurora.client.api.sla import JobUpTimeLimit
 from apache.aurora.client.base import (
+    AURORA_ADMIN_USER_AGENT_NAME,
     check_and_log_response,
     combine_messages,
     die,
@@ -42,6 +42,7 @@ from apache.aurora.client.base import (
     GROUPING_OPTION,
     requires
 )
+from apache.aurora.client.factory import make_client
 from apache.aurora.common.aurora_job_key import AuroraJobKey
 from apache.aurora.common.clusters import CLUSTERS
 from apache.aurora.common.shellify import shellify
@@ -61,6 +62,10 @@ MIN_SLA_INSTANCE_COUNT = optparse.Option(
 )
 
 
+def make_admin_client(cluster):
+  return make_client(cluster, AURORA_ADMIN_USER_AGENT_NAME)
+
+
 @app.command
 @app.command_option('--force', dest='force', default=False, action='store_true',
     help='Force expensive queries to run.')
@@ -142,7 +147,8 @@ def query(args, options):
   if not (states <= ACTIVE_STATES) and not options.force:
     die('--force is required for expensive queries (states outside ACTIVE states')
 
-  api = AuroraClientAPI(CLUSTERS[cluster], options.verbosity)
+  api = make_admin_client(cluster)
+
   query_info = api.query(TaskQuery(role=role, jobName=job, instanceIds=instances, statuses=states))
   if query_info.responseCode != ResponseCode.OK:
     die('Failed to query scheduler: %s' % combine_messages(query_info))
@@ -181,8 +187,7 @@ def set_quota(cluster, role, cpu_str, ram, disk):
   except ValueError as e:
     die(str(e))
 
-  options = app.get_options()
-  resp = AuroraClientAPI(CLUSTERS[cluster], options.verbosity).set_quota(role, cpu, ram_mb, disk_mb)
+  resp = make_admin_client(cluster).set_quota(role, cpu, ram_mb, disk_mb)
   check_and_log_response(resp)
 
 
@@ -197,8 +202,7 @@ def increase_quota(cluster, role, cpu_str, ram_str, disk_str):
   ram = parse_data(ram_str)
   disk = parse_data(disk_str)
 
-  options = app.get_options()
-  client = AuroraClientAPI(CLUSTERS[cluster], options.verbosity == 'verbose')
+  client = make_admin_client(cluster)
   resp = client.get_quota(role)
   quota = resp.result.getQuotaResult.quota
   log.info('Current quota for %s:\n\tCPU\t%s\n\tRAM\t%s MB\n\tDisk\t%s MB' %
@@ -222,8 +226,7 @@ def scheduler_backup_now(cluster):
 
   Immediately initiates a full storage backup.
   """
-  options = app.get_options()
-  check_and_log_response(AuroraClientAPI(CLUSTERS[cluster], options.verbosity).perform_backup())
+  check_and_log_response(make_admin_client(cluster).perform_backup())
 
 
 @app.command
@@ -233,8 +236,7 @@ def scheduler_list_backups(cluster):
 
   Lists backups available for recovery.
   """
-  options = app.get_options()
-  resp = AuroraClientAPI(CLUSTERS[cluster], options.verbosity).list_backups()
+  resp = make_admin_client(cluster).list_backups()
   check_and_log_response(resp)
   backups = resp.result.listBackupsResult.backups
   print('%s available backups:' % len(backups))
@@ -249,9 +251,7 @@ def scheduler_stage_recovery(cluster, backup_id):
 
   Stages a backup for recovery.
   """
-  options = app.get_options()
-  check_and_log_response(
-      AuroraClientAPI(CLUSTERS[cluster], options.verbosity).stage_recovery(backup_id))
+  check_and_log_response(make_admin_client(cluster).stage_recovery(backup_id))
 
 
 @app.command
@@ -261,8 +261,7 @@ def scheduler_print_recovery_tasks(cluster):
 
   Prints all active tasks in a staged recovery.
   """
-  options = app.get_options()
-  resp = AuroraClientAPI(CLUSTERS[cluster], options.verbosity).query_recovery(
+  resp = make_admin_client(cluster).query_recovery(
       TaskQuery(statuses=ACTIVE_STATES))
   check_and_log_response(resp)
   log.info('Role\tJob\tShard\tStatus\tTask ID')
@@ -284,9 +283,7 @@ def scheduler_delete_recovery_tasks(cluster, task_ids):
   Deletes a comma-separated list of task IDs from a staged recovery.
   """
   ids = set(task_ids.split(','))
-  options = app.get_options()
-  check_and_log_response(AuroraClientAPI(CLUSTERS[cluster], options.verbosity)
-      .delete_recovery_tasks(TaskQuery(taskIds=ids)))
+  check_and_log_response(make_admin_client(cluster).delete_recovery_tasks(TaskQuery(taskIds=ids)))
 
 
 @app.command
@@ -296,9 +293,7 @@ def scheduler_commit_recovery(cluster):
 
   Commits a staged recovery.
   """
-  options = app.get_options()
-  check_and_log_response(AuroraClientAPI(CLUSTERS[cluster], options.verbosity)
-      .commit_recovery())
+  check_and_log_response(make_admin_client(cluster).commit_recovery())
 
 
 @app.command
@@ -308,9 +303,7 @@ def scheduler_unload_recovery(cluster):
 
   Unloads a staged recovery.
   """
-  options = app.get_options()
-  check_and_log_response(AuroraClientAPI(CLUSTERS[cluster], options.verbosity)
-      .unload_recovery())
+  check_and_log_response(make_admin_client(cluster).unload_recovery())
 
 
 @app.command
@@ -320,8 +313,7 @@ def scheduler_snapshot(cluster):
 
   Request that the scheduler perform a storage snapshot and block until complete.
   """
-  options = app.get_options()
-  check_and_log_response(AuroraClientAPI(CLUSTERS[cluster], options.verbosity).snapshot())
+  check_and_log_response(make_admin_client(cluster).snapshot())
 
 
 @app.command
@@ -331,8 +323,7 @@ def get_locks(cluster):
 
   Prints all context/operation locks in the scheduler.
   """
-  options = app.get_options()
-  resp = AuroraClientAPI(CLUSTERS[cluster], options.verbosity).get_locks()
+  resp = make_admin_client(cluster).get_locks()
   check_and_log_response(resp)
 
   pp = pprint.PrettyPrinter(indent=2)
@@ -425,9 +416,9 @@ def sla_list_safe_domain(cluster, percentage, duration):
   override_jobs = parse_jobs_file(options.override_filename) if options.override_filename else {}
   get_grouping_or_die(options.grouping)
 
-  vector = AuroraClientAPI(
-      CLUSTERS[cluster],
-      options.verbosity).sla_get_safe_domain_vector(options.min_instance_count, include_hosts)
+  vector = make_admin_client(cluster).sla_get_safe_domain_vector(
+      options.min_instance_count,
+      include_hosts)
   groups = vector.get_safe_hosts(sla_percentage, sla_duration.as_(Time.SECONDS),
       override_jobs, options.grouping)
 
@@ -482,9 +473,7 @@ def sla_probe_hosts(cluster, percentage, duration):
   hosts = parse_hostnames(options.filename, options.hosts)
   get_grouping_or_die(options.grouping)
 
-  vector = AuroraClientAPI(
-      CLUSTERS[cluster],
-      options.verbosity).sla_get_safe_domain_vector(options.min_instance_count, hosts)
+  vector = make_admin_client(cluster).sla_get_safe_domain_vector(options.min_instance_count, hosts)
   groups = vector.probe_hosts(sla_percentage, sla_duration.as_(Time.SECONDS), options.grouping)
 
   output, _ = format_sla_results(groups)
@@ -520,7 +509,5 @@ def get_scheduler(cluster):
 
   Dumps the leading scheduler endpoint URL.
   """
-  options = app.get_options()
-  print("Found leading scheduler at: %s" % AuroraClientAPI(
-      CLUSTERS[cluster],
-      options.verbosity).scheduler_proxy.scheduler_client().raw_url)
+  print("Found leading scheduler at: %s" %
+      make_admin_client(cluster).scheduler_proxy.scheduler_client().raw_url)

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8c841060/src/main/python/apache/aurora/client/commands/core.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/commands/core.py b/src/main/python/apache/aurora/client/commands/core.py
index 7b970e8..cbaa67a 100644
--- a/src/main/python/apache/aurora/client/commands/core.py
+++ b/src/main/python/apache/aurora/client/commands/core.py
@@ -33,13 +33,15 @@ from apache.aurora.client.api.job_monitor import JobMonitor
 from apache.aurora.client.api.quota_check import print_quota
 from apache.aurora.client.api.updater_util import UpdaterConfig
 from apache.aurora.client.base import (
+    AURORA_V1_USER_AGENT_NAME,
     check_and_log_response,
     combine_messages,
     deprecation_warning,
     die,
     handle_open,
     requires,
-    synthesize_url
+    synthesize_url,
+    user_agent
 )
 from apache.aurora.client.config import get_config, GlobalHookRegistry
 from apache.aurora.client.factory import make_client, make_client_factory
@@ -66,6 +68,14 @@ from gen.apache.aurora.api.constants import ACTIVE_STATES, AURORA_EXECUTOR_NAME,
 from gen.apache.aurora.api.ttypes import ExecutorConfig, ResponseCode, ScheduleStatus
 
 
+def make_v1_client_factory():
+  return make_client_factory(user_agent(AURORA_V1_USER_AGENT_NAME))
+
+
+def make_v1_client(cluster):
+  return make_client(cluster, AURORA_V1_USER_AGENT_NAME)
+
+
 class CoreCommandHook(object):
   """Limited version of the command hooks framework ported to clientv1 commands.
   Core command hooks can only be created by invoking "CoreCommandHook.register_hook"
@@ -196,7 +206,7 @@ def really_create(job_spec, config_file, options):
   except ValueError as v:
     print("Error: %s" % v)
     sys.exit(1)
-  api = make_client(config.cluster())
+  api = make_v1_client(config.cluster())
   resp = api.create_job(config)
   check_and_log_response(resp)
   handle_open(api.scheduler_proxy.scheduler_client().url, config.role(), config.environment(),
@@ -267,7 +277,7 @@ def diff(job_spec, config_file):
     role = config.role()
     env = config.environment()
     name = config.name()
-  api = make_client(cluster)
+  api = make_v1_client(cluster)
   resp = api.query(api.build_query(role, name, statuses=ACTIVE_STATES, env=env))
   if resp.responseCode != ResponseCode.OK:
     die('Request failed, server responded with "%s"' % combine_messages(resp))
@@ -336,7 +346,7 @@ def do_open(args, _):
   if not cluster_name:
     die('cluster is required')
 
-  api = make_client(cluster_name)
+  api = make_v1_client(cluster_name)
 
   import webbrowser
   webbrowser.open_new_tab(
@@ -417,7 +427,7 @@ def inspect(job_spec, config_file):
 
 def really_start_cron(args, options):
   api, job_key, config_file = LiveJobDisambiguator.disambiguate_args_or_die(
-      args, options, make_client_factory())
+      args, options, make_v1_client_factory())
   config = get_job_config(job_key.to_path(), config_file, options) if config_file else None
   resp = api.start_cronjob(job_key, config=config)
   check_and_log_response(resp)
@@ -488,7 +498,7 @@ def list_jobs(cluster_and_role):
   if cluster_and_role.count('/') != 1:
     die('list_jobs parameter must be in cluster/role format')
   cluster, role = cluster_and_role.split('/')
-  api = make_client(cluster)
+  api = make_v1_client(cluster)
   resp = api.get_jobs(role)
   check_and_log_response(resp)
   for job in resp.result.getJobsResult.configs:
@@ -518,7 +528,7 @@ def really_kill(args, options):
     print('Shards option is required for kill; use killall to kill all shards', file=sys.stderr)
     exit(1)
   api, job_key, config_file = LiveJobDisambiguator.disambiguate_args_or_die(
-      args, options, make_client_factory())
+      args, options, make_v1_client_factory())
   instance_key = str(job_key)
   if options.shards is not None:
     instance_key = "%s/%s" % (instance_key, ",".join(map(str, options.shards)))
@@ -602,7 +612,7 @@ def really_killall(args, options):
   v1_deprecation_warning("killall", new_cmd)
 
   config = get_job_config(job_key.to_path(), config_file, options) if config_file else None
-  api = make_client(job_key.cluster)
+  api = make_v1_client(job_key.cluster)
   if options.batch_size is not None:
     kill_in_batches(api, job_key, None, options.batch_size, options.max_failures_option)
   else:
@@ -675,7 +685,7 @@ def status(args, options):
               taskString))
 
   api, job_key, _ = LiveJobDisambiguator.disambiguate_args_or_die(
-      args, options, make_client_factory())
+      args, options, make_v1_client_factory())
   v1_deprecation_warning("status", ["job", "status", args[0]])
   resp = api.check_status(job_key)
   check_and_log_response(resp)
@@ -713,7 +723,7 @@ def really_update(job_spec, config_file, options):
 
   maybe_disable_hooks(options)
   config = get_job_config(job_spec, config_file, options)
-  api = make_client(config.cluster())
+  api = make_v1_client(config.cluster())
   if not options.force:
     warn_if_dangerous_change(api, job_spec, config)
   resp = api.update_job(config, options.health_check_interval_seconds, options.shards)
@@ -775,7 +785,7 @@ def really_restart(args, options):
     exit(1)
   maybe_disable_hooks(options)
   api, job_key, config_file = LiveJobDisambiguator.disambiguate_args_or_die(
-      args, options, make_client_factory())
+      args, options, make_v1_client_factory())
   config = get_job_config(job_key.to_path(), config_file, options) if config_file else None
   updater_config = UpdaterConfig(
       options.batch_size,
@@ -848,7 +858,7 @@ def restart(args, options):
 
 def really_cancel_update(args, options):
   api, job_key, config_file = LiveJobDisambiguator.disambiguate_args_or_die(
-      args, options, make_client_factory())
+      args, options, make_v1_client_factory())
   new_cmd = ["job", "cancel-update", str(job_key)]
   v1_deprecation_warning("cancel_update", new_cmd)
   config = get_job_config(job_key.to_path(), config_file, options) if config_file else None
@@ -881,7 +891,7 @@ def get_quota(role):
   """
   options = app.get_options()
   v1_deprecation_warning("get_quota", ["quota", "get", "%s/%s" % (options.cluster, role)])
-  resp = make_client(options.cluster).get_quota(role)
+  resp = make_v1_client(options.cluster).get_quota(role)
   quota_result = resp.result.getQuotaResult
   print_quota(quota_result.quota, 'Total allocated quota', role)
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8c841060/src/main/python/apache/aurora/client/commands/ssh.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/commands/ssh.py b/src/main/python/apache/aurora/client/commands/ssh.py
index b0916ed..ab6832c 100644
--- a/src/main/python/apache/aurora/client/commands/ssh.py
+++ b/src/main/python/apache/aurora/client/commands/ssh.py
@@ -17,7 +17,7 @@ import subprocess
 from twitter.common import app
 
 from apache.aurora.client.api.command_runner import DistributedCommandRunner
-from apache.aurora.client.base import check_and_log_response, die
+from apache.aurora.client.base import AURORA_V1_USER_AGENT_NAME, check_and_log_response, die
 from apache.aurora.client.factory import make_client
 from apache.aurora.client.options import EXECUTOR_SANDBOX_OPTION, SSH_USER_OPTION
 from apache.aurora.common.aurora_job_key import AuroraJobKey
@@ -68,7 +68,7 @@ def ssh(args, options):
     newcmd.append("--command=\"%s\"" % " ".join(args))
   v1_deprecation_warning("ssh", newcmd)
 
-  api = make_client(cluster_name)
+  api = make_client(cluster_name, AURORA_V1_USER_AGENT_NAME)
   resp = api.query(api.build_query(role, name, set([int(shard)]), env=env))
   check_and_log_response(resp)
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8c841060/src/main/python/apache/aurora/client/factory.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/factory.py b/src/main/python/apache/aurora/client/factory.py
index 4b31e85..85a1398 100644
--- a/src/main/python/apache/aurora/client/factory.py
+++ b/src/main/python/apache/aurora/client/factory.py
@@ -26,16 +26,16 @@ from .base import die
 # TODO(wickman) Kill make_client and make_client_factory as part of MESOS-3801.
 # These are currently necessary indirections for the LiveJobDisambiguator among
 # other things but can go away once those are scrubbed.
-def make_client_factory():
+def make_client_factory(user_agent):
   verbose = getattr(app.get_options(), 'verbosity', 'normal') == 'verbose'
   class TwitterAuroraClientAPI(HookedAuroraClientAPI):
     def __init__(self, cluster, *args, **kw):
       if cluster not in CLUSTERS:
         die('Unknown cluster: %s' % cluster)
       super(TwitterAuroraClientAPI, self).__init__(CLUSTERS[cluster], *args, **kw)
-  return functools.partial(TwitterAuroraClientAPI, verbose=verbose)
+  return functools.partial(TwitterAuroraClientAPI, user_agent=user_agent, verbose=verbose)
 
 
-def make_client(cluster):
-  factory = make_client_factory()
+def make_client(cluster, user_agent):
+  factory = make_client_factory(user_agent)
   return factory(cluster.name if isinstance(cluster, Cluster) else cluster)

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8c841060/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 76e079a..a5fd7c0 100644
--- a/src/main/python/apache/aurora/common/transport.py
+++ b/src/main/python/apache/aurora/common/transport.py
@@ -25,16 +25,24 @@ except ImportError:
   from urllib.parse import urlparse
 
 
+DEFAULT_USER_AGENT = 'Python TRequestsTransport v1.0'
+
+
 def default_requests_session_factory():
   session = requests.session()
-  session.headers['User-Agent'] = 'Python TRequestsTransport v1.0'
   return session
 
 
 class TRequestsTransport(TTransportBase):
   """A Thrift HTTP client based upon the requests module."""
 
-  def __init__(self, uri, auth=None, session_factory=default_requests_session_factory):
+  def __init__(
+      self,
+      uri,
+      auth=None,
+      session_factory=default_requests_session_factory,
+      user_agent=DEFAULT_USER_AGENT):
+
     """Construct a TRequestsTransport.
 
     Construct a Thrift transport based upon the requests module.  URI is the
@@ -46,11 +54,14 @@ 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.
     """
-    self.__session = None
+    self._session = None
     self.__session_factory = session_factory
     if not callable(session_factory):
       raise TypeError('session_factory should be a callable that produces a requests.Session!')
+    self.__user_agent = user_agent
     self.__wbuf = BytesIO()
     self.__rbuf = BytesIO()
     self.__uri = uri
@@ -65,13 +76,18 @@ class TRequestsTransport(TTransportBase):
     logging.getLogger('requests').setLevel(logging.WARNING)
 
   def isOpen(self):
-    return self.__session is not None
+    return self._session is not None
 
   def open(self):
-    self.__session = self.__session_factory()
+    session = self.__session_factory()
+    requests_default_agent = requests.utils.default_user_agent()
+    if session.headers.get('User-Agent', requests_default_agent) == requests_default_agent:
+      session.headers['User-Agent'] = self.__user_agent
+
+    self._session = session
 
   def close(self):
-    session, self.__session = self.__session, None
+    session, self._session = self._session, None
     session.close()
 
   def setTimeout(self, ms):
@@ -92,12 +108,12 @@ class TRequestsTransport(TTransportBase):
     data = self.__wbuf.getvalue()
     self.__wbuf = BytesIO()
 
-    self.__session.headers['Content-Type'] = 'application/x-thrift'
-    self.__session.headers['Content-Length'] = str(len(data))
-    self.__session.headers['Host'] = self.__urlparse.hostname
+    self._session.headers['Content-Type'] = 'application/x-thrift'
+    self._session.headers['Content-Length'] = str(len(data))
+    self._session.headers['Host'] = self.__urlparse.hostname
 
     try:
-      response = self.__session.post(
+      response = self._session.post(
           self.__uri,
           data=data,
           timeout=self.__timeout,

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8c841060/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 b3232e9..a3a40b7 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
@@ -354,7 +354,9 @@ def test_url_when_not_connected_and_cluster_has_no_proxy_url(scheme):
   service_endpoints = [ServiceInstance.unpack(service_json)]
 
   def make_mock_client(proxy_url):
-    client = scheduler_client.ZookeeperSchedulerClient(Cluster(proxy_url=proxy_url),
+    client = scheduler_client.ZookeeperSchedulerClient(
+        Cluster(proxy_url=proxy_url),
+        user_agent='Some-User-Agent',
         _deadline=lambda x, **kws: x())
     client.get_scheduler_serverset = mock.MagicMock(return_value=(mock_zk, service_endpoints))
     client.SERVERSET_TIMEOUT = Amount(0, Time.SECONDS)
@@ -384,14 +386,30 @@ def test_url_when_not_connected_and_cluster_has_no_proxy_url(scheme):
 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)
-  scheduler_client.SchedulerClient._connect_scheduler(
-      'https://scheduler.example.com:1337',
-      mock_time)
-  assert mock_client.return_value.open.call_count == 2
+
+  client = scheduler_client.SchedulerClient('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())
   mock_time.sleep.assert_called_once_with(
       scheduler_client.SchedulerClient.RETRY_TIMEOUT.as_(Time.SECONDS))
 
 
+@mock.patch('apache.aurora.client.api.scheduler_client.TRequestsTransport', spec=TRequestsTransport)
+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)
+
+  user_agent = 'Some-User-Agent'
+
+  client = scheduler_client.SchedulerClient(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.patch('apache.aurora.client.api.scheduler_client.SchedulerClient',
             spec=scheduler_client.SchedulerClient)
 @mock.patch('threading._Event.wait')

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8c841060/src/test/python/apache/aurora/client/cli/test_diff.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/test_diff.py b/src/test/python/apache/aurora/client/cli/test_diff.py
index 95c7c92..d8da68f 100644
--- a/src/test/python/apache/aurora/client/cli/test_diff.py
+++ b/src/test/python/apache/aurora/client/cli/test_diff.py
@@ -77,6 +77,7 @@ class TestDiffCommand(AuroraClientCommandTest):
         patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS),
         patch('subprocess.call', return_value=0),
         patch('json.loads', return_value=Mock())) as (_, _, subprocess_patch, _):
+
       mock_scheduler_proxy.getTasksStatus.return_value = self.create_status_response()
       self.setup_populate_job_config(mock_scheduler_proxy)
       with temporary_file() as fp:
@@ -142,6 +143,7 @@ class TestDiffCommand(AuroraClientCommandTest):
             options,
             subprocess_patch,
             json_patch):
+
       with temporary_file() as fp:
         fp.write(self.get_valid_config())
         fp.flush()

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8c841060/src/test/python/apache/aurora/client/commands/test_admin.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/commands/test_admin.py b/src/test/python/apache/aurora/client/commands/test_admin.py
index c7d970c8..b093682 100644
--- a/src/test/python/apache/aurora/client/commands/test_admin.py
+++ b/src/test/python/apache/aurora/client/commands/test_admin.py
@@ -91,28 +91,32 @@ class TestQueryCommand(AuroraClientCommandTest):
   def test_query(self):
     """Tests successful execution of the query command."""
     mock_options = self.setup_mock_options(force=True, shards="0")
-    mock_api, mock_scheduler_proxy = self.create_mock_api()
     with contextlib.nested(
         patch('twitter.common.app.get_options', return_value=mock_options),
-        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
-        patch('apache.aurora.client.commands.admin.CLUSTERS', new=self.TEST_CLUSTERS)):
+        patch('apache.aurora.client.commands.admin.make_admin_client',
+            return_value=create_autospec(spec=AuroraClientAPI)),
+        patch('apache.aurora.client.commands.admin.CLUSTERS', new=self.TEST_CLUSTERS)
+    ) as (_, mock_make_admin_client, _):
 
-      mock_scheduler_proxy.getTasksStatus.return_value = self.create_response(self.create_task())
+      api = mock_make_admin_client.return_value
+      api.query.return_value = self.create_response(self.create_task())
 
       query([self.TEST_CLUSTER, 'test_role', 'test_job'], mock_options)
 
-      mock_scheduler_proxy.getTasksStatus.assert_called_with(self.task_query())
+      api.query.assert_called_with(self.task_query())
 
   def test_query_fails(self):
     """Tests failed execution of the query command."""
     mock_options = self.setup_mock_options(shards="0")
-    mock_api, mock_scheduler_proxy = self.create_mock_api()
     with contextlib.nested(
         patch('twitter.common.app.get_options', return_value=mock_options),
-        patch('apache.aurora.client.api.SchedulerProxy', return_value=mock_scheduler_proxy),
-        patch('apache.aurora.client.commands.admin.CLUSTERS', new=self.TEST_CLUSTERS)):
+        patch('apache.aurora.client.commands.admin.make_admin_client',
+            return_value=create_autospec(spec=AuroraClientAPI)),
+        patch('apache.aurora.client.commands.admin.CLUSTERS', new=self.TEST_CLUSTERS)
+    ) as (_, mock_make_admin_client, _):
 
-      mock_scheduler_proxy.getTasksStatus.return_value = self.create_response(self.create_task())
+      api = mock_make_admin_client.return_value
+      api.query.return_value = self.create_response(self.create_task())
 
       try:
         query([self.TEST_CLUSTER, 'test_role', 'test_job'], mock_options)
@@ -121,7 +125,7 @@ class TestQueryCommand(AuroraClientCommandTest):
       else:
         assert 'Expected exception is not raised'
 
-      mock_scheduler_proxy.getTasksStatus.assert_called_with(self.task_query())
+      api.query.assert_called_with(self.task_query())
 
 
 class TestIncreaseQuotaCommand(AuroraClientCommandTest):
@@ -140,25 +144,26 @@ class TestIncreaseQuotaCommand(AuroraClientCommandTest):
     mock_options = self.setup_mock_options()
     with contextlib.nested(
         patch('twitter.common.app.get_options', return_value=mock_options),
-        patch('apache.aurora.client.commands.admin.AuroraClientAPI',
-            new=create_autospec(spec=AuroraClientAPI)),
+        patch('apache.aurora.client.commands.admin.make_admin_client',
+            return_value=create_autospec(spec=AuroraClientAPI)),
         patch('apache.aurora.client.commands.admin.CLUSTERS', new=self.TEST_CLUSTERS)
-    ) as (_, api, _):
+    ) as (_, mock_make_admin_client, _):
 
+      api = mock_make_admin_client.return_value
       role = 'test_role'
-      api.return_value.get_quota.return_value = self.create_response(
+      api.get_quota.return_value = self.create_response(
           ResourceAggregate(20.0, 4000, 6000),
           ResourceAggregate(15.0, 2000, 3000),
           ResourceAggregate(6.0, 200, 600),
       )
-      api.return_value.set_quota.return_value = self.create_simple_success_response()
+      api.set_quota.return_value = self.create_simple_success_response()
 
       increase_quota([self.TEST_CLUSTER, role, '4.0', '1MB', '1MB'])
 
-      api.return_value.set_quota.assert_called_with(role, 24.0, 4001, 6001)
-      assert type(api.return_value.set_quota.call_args[0][1]) == type(float())
-      assert type(api.return_value.set_quota.call_args[0][2]) == type(int())
-      assert type(api.return_value.set_quota.call_args[0][3]) == type(int())
+      api.set_quota.assert_called_with(role, 24.0, 4001, 6001)
+      assert type(api.set_quota.call_args[0][1]) == type(float())
+      assert type(api.set_quota.call_args[0][2]) == type(int())
+      assert type(api.set_quota.call_args[0][3]) == type(int())
 
 
 class TestSetQuotaCommand(AuroraClientCommandTest):
@@ -176,20 +181,21 @@ class TestSetQuotaCommand(AuroraClientCommandTest):
     mock_options = self.setup_mock_options()
     with contextlib.nested(
         patch('twitter.common.app.get_options', return_value=mock_options),
-        patch('apache.aurora.client.commands.admin.AuroraClientAPI',
-              new=create_autospec(spec=AuroraClientAPI)),
+        patch('apache.aurora.client.commands.admin.make_admin_client',
+              return_value=create_autospec(spec=AuroraClientAPI)),
         patch('apache.aurora.client.commands.admin.CLUSTERS', new=self.TEST_CLUSTERS)
-    ) as (_, api, _):
+    ) as (_, mock_make_admin_client, _):
 
+      api = mock_make_admin_client.return_value
       role = 'test_role'
-      api.return_value.set_quota.return_value = self.create_simple_success_response()
+      api.set_quota.return_value = self.create_simple_success_response()
 
       set_quota([self.TEST_CLUSTER, role, '4.0', '10MB', '10MB'])
 
-      api.return_value.set_quota.assert_called_with(role, 4.0, 10, 10)
-      assert type(api.return_value.set_quota.call_args[0][1]) == type(float())
-      assert type(api.return_value.set_quota.call_args[0][2]) == type(int())
-      assert type(api.return_value.set_quota.call_args[0][3]) == type(int())
+      api.set_quota.assert_called_with(role, 4.0, 10, 10)
+      assert type(api.set_quota.call_args[0][1]) == type(float())
+      assert type(api.set_quota.call_args[0][2]) == type(int())
+      assert type(api.set_quota.call_args[0][3]) == type(int())
 
 
 class TestGetLocksCommand(AuroraClientCommandTest):
@@ -215,17 +221,18 @@ class TestGetLocksCommand(AuroraClientCommandTest):
     mock_options = self.setup_mock_options()
     with contextlib.nested(
         patch('twitter.common.app.get_options', return_value=mock_options),
-        patch('apache.aurora.client.commands.admin.AuroraClientAPI',
-              new=create_autospec(spec=AuroraClientAPI)),
+        patch('apache.aurora.client.commands.admin.make_admin_client',
+              return_value=create_autospec(spec=AuroraClientAPI)),
         patch('apache.aurora.client.commands.admin.CLUSTERS', new=self.TEST_CLUSTERS),
         patch('apache.aurora.client.commands.admin.print_results'),
-    ) as (_, api, _, mock_print_results):
+    ) as (_, mock_make_admin_client, _, mock_print_results):
 
-      api.return_value.get_locks.return_value = self.create_response(self.LOCKS)
+      api = mock_make_admin_client.return_value
+      api.get_locks.return_value = self.create_response(self.LOCKS)
 
       get_locks([self.TEST_CLUSTER])
 
-      assert api.return_value.get_locks.call_count == 1
+      assert api.get_locks.call_count == 1
       assert mock_print_results.call_count == 1
       assert "'message': '%s'" % self.MESSAGE in mock_print_results.call_args[0][0][0]
       assert "'user': '%s'" % self.USER in mock_print_results.call_args[0][0][0]
@@ -244,12 +251,13 @@ class TestGetSchedulerCommand(AuroraClientCommandTest):
 
     with contextlib.nested(
         patch('twitter.common.app.get_options', return_value=mock_options),
-        patch('apache.aurora.client.commands.admin.AuroraClientAPI',
-              new=create_autospec(spec=AuroraClientAPI)),
+        patch('apache.aurora.client.commands.admin.make_admin_client',
+              return_value=create_autospec(spec=AuroraClientAPI)),
         patch('apache.aurora.client.commands.admin.CLUSTERS', new=self.TEST_CLUSTERS),
-    ) as (_, api, _):
+    ) as (_, mock_make_admin_client, _):
 
-      api.return_value.scheduler_proxy = PropertyMock(return_value=mock_proxy)
+      api = mock_make_admin_client.return_value
+      api.scheduler_proxy = PropertyMock(return_value=mock_proxy)
 
       get_scheduler([self.TEST_CLUSTER])
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8c841060/src/test/python/apache/aurora/client/commands/test_admin_sla.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/commands/test_admin_sla.py b/src/test/python/apache/aurora/client/commands/test_admin_sla.py
index 74fc79c..ec558f8 100644
--- a/src/test/python/apache/aurora/client/commands/test_admin_sla.py
+++ b/src/test/python/apache/aurora/client/commands/test_admin_sla.py
@@ -70,7 +70,7 @@ class TestAdminSlaListSafeDomainCommand(AuroraClientCommandTest):
     mock_options = self.setup_mock_options()
     mock_vector = self.create_mock_vector(self.create_hosts(3, 80, 100))
     with contextlib.nested(
-        patch('apache.aurora.client.commands.admin.AuroraClientAPI',
+        patch('apache.aurora.client.commands.admin.make_client',
             new=create_autospec(spec=AuroraClientAPI)),
         patch('apache.aurora.client.commands.admin.print_results'),
         patch('apache.aurora.client.commands.admin.CLUSTERS', new=self.TEST_CLUSTERS),
@@ -95,7 +95,7 @@ class TestAdminSlaListSafeDomainCommand(AuroraClientCommandTest):
       fp.flush()
       mock_options = self.setup_mock_options(exclude=fp.name)
       with contextlib.nested(
-          patch('apache.aurora.client.commands.admin.AuroraClientAPI',
+          patch('apache.aurora.client.commands.admin.make_client',
               new=create_autospec(spec=AuroraClientAPI)),
           patch('apache.aurora.client.commands.admin.print_results'),
           patch('apache.aurora.client.commands.admin.CLUSTERS', new=self.TEST_CLUSTERS),
@@ -118,7 +118,7 @@ class TestAdminSlaListSafeDomainCommand(AuroraClientCommandTest):
     mock_vector = self.create_mock_vector(self.create_hosts(3, 80, 100))
     mock_options = self.setup_mock_options(exclude_list=','.join(['h0', 'h1']))
     with contextlib.nested(
-        patch('apache.aurora.client.commands.admin.AuroraClientAPI',
+        patch('apache.aurora.client.commands.admin.make_client',
             new=create_autospec(spec=AuroraClientAPI)),
         patch('apache.aurora.client.commands.admin.print_results'),
         patch('apache.aurora.client.commands.admin.CLUSTERS', new=self.TEST_CLUSTERS),
@@ -145,7 +145,7 @@ class TestAdminSlaListSafeDomainCommand(AuroraClientCommandTest):
       fp.flush()
       mock_options = self.setup_mock_options(include=fp.name)
       with contextlib.nested(
-          patch('apache.aurora.client.commands.admin.AuroraClientAPI',
+          patch('apache.aurora.client.commands.admin.make_client',
               new=create_autospec(spec=AuroraClientAPI)),
           patch('apache.aurora.client.commands.admin.print_results'),
           patch('apache.aurora.client.commands.admin.CLUSTERS', new=self.TEST_CLUSTERS),
@@ -171,7 +171,7 @@ class TestAdminSlaListSafeDomainCommand(AuroraClientCommandTest):
     hosts = ['h0', 'h1']
     mock_options = self.setup_mock_options(include_list=','.join(hosts))
     with contextlib.nested(
-        patch('apache.aurora.client.commands.admin.AuroraClientAPI',
+        patch('apache.aurora.client.commands.admin.make_client',
             new=create_autospec(spec=AuroraClientAPI)),
         patch('apache.aurora.client.commands.admin.print_results'),
         patch('apache.aurora.client.commands.admin.CLUSTERS', new=self.TEST_CLUSTERS),
@@ -199,7 +199,7 @@ class TestAdminSlaListSafeDomainCommand(AuroraClientCommandTest):
       fp.flush()
       mock_options = self.setup_mock_options(override=fp.name)
       with contextlib.nested(
-          patch('apache.aurora.client.commands.admin.AuroraClientAPI',
+          patch('apache.aurora.client.commands.admin.make_client',
               new=create_autospec(spec=AuroraClientAPI)),
           patch('apache.aurora.client.commands.admin.print_results'),
           patch('apache.aurora.client.commands.admin.CLUSTERS', new=self.TEST_CLUSTERS),
@@ -224,7 +224,7 @@ class TestAdminSlaListSafeDomainCommand(AuroraClientCommandTest):
     mock_options = self.setup_mock_options(list_jobs=True)
     mock_vector = self.create_mock_vector(self.create_hosts(3, 50, 100))
     with contextlib.nested(
-        patch('apache.aurora.client.commands.admin.AuroraClientAPI',
+        patch('apache.aurora.client.commands.admin.make_client',
             new=create_autospec(spec=AuroraClientAPI)),
         patch('apache.aurora.client.commands.admin.print_results'),
         patch('apache.aurora.client.commands.admin.CLUSTERS', new=self.TEST_CLUSTERS),
@@ -331,7 +331,7 @@ class TestAdminSlaProbeHostsCommand(AuroraClientCommandTest):
     mock_options = self.setup_mock_options(hosts=','.join(hosts))
     mock_vector = self.create_mock_probe_hosts_vector([self.create_probe_hosts(2, 80, True, 0)])
     with contextlib.nested(
-        patch('apache.aurora.client.commands.admin.AuroraClientAPI',
+        patch('apache.aurora.client.commands.admin.make_client',
             new=create_autospec(spec=AuroraClientAPI)),
         patch('apache.aurora.client.commands.admin.print_results'),
         patch('apache.aurora.client.commands.admin.CLUSTERS', new=self.TEST_CLUSTERS),
@@ -361,7 +361,7 @@ class TestAdminSlaProbeHostsCommand(AuroraClientCommandTest):
       fp.flush()
       mock_options = self.setup_mock_options(filename=fp.name)
       with contextlib.nested(
-          patch('apache.aurora.client.commands.admin.AuroraClientAPI',
+          patch('apache.aurora.client.commands.admin.make_client',
               new=create_autospec(spec=AuroraClientAPI)),
           patch('apache.aurora.client.commands.admin.print_results'),
           patch('apache.aurora.client.commands.admin.CLUSTERS', new=self.TEST_CLUSTERS),

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8c841060/src/test/python/apache/aurora/client/commands/test_create.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/commands/test_create.py b/src/test/python/apache/aurora/client/commands/test_create.py
index d7c32b3..2a61d6e 100644
--- a/src/test/python/apache/aurora/client/commands/test_create.py
+++ b/src/test/python/apache/aurora/client/commands/test_create.py
@@ -110,7 +110,7 @@ class TestClientCreateCommand(AuroraClientCommandTest):
     # Next, create gets an API object via make_client. We need to replace that with a mock API.
     (mock_api, mock_scheduler_proxy) = self.create_mock_api()
     with contextlib.nested(
-        patch('apache.aurora.client.commands.core.make_client', return_value=mock_api),
+        patch('apache.aurora.client.commands.core.make_v1_client', return_value=mock_api),
         patch('twitter.common.app.get_options', return_value=mock_options)) as (make_client,
         options):
 
@@ -149,7 +149,7 @@ class TestClientCreateCommand(AuroraClientCommandTest):
     (mock_api, mock_scheduler_proxy) = self.create_mock_api()
     with contextlib.nested(
         patch('threading._Event.wait'),
-        patch('apache.aurora.client.commands.core.make_client', return_value=mock_api),
+        patch('apache.aurora.client.commands.core.make_v1_client', return_value=mock_api),
         patch('twitter.common.app.get_options', return_value=mock_options)) as (sleep, make_client,
         options):
       mock_query = self.create_mock_query()
@@ -181,7 +181,7 @@ class TestClientCreateCommand(AuroraClientCommandTest):
     mock_options = self.setup_mock_options()
     (mock_api, mock_scheduler_proxy) = self.create_mock_api()
     with contextlib.nested(
-        patch('apache.aurora.client.commands.core.make_client', return_value=mock_api),
+        patch('apache.aurora.client.commands.core.make_v1_client', return_value=mock_api),
         patch('twitter.common.app.get_options', return_value=mock_options)) as (make_client,
         options):
       mock_api.create_job.return_value = self.get_failed_createjob_response()
@@ -206,7 +206,7 @@ class TestClientCreateCommand(AuroraClientCommandTest):
     (mock_api, mock_scheduler_proxy) = self.create_mock_api()
     with contextlib.nested(
         patch('threading._Event.wait'),
-        patch('apache.aurora.client.commands.core.make_client', return_value=mock_api),
+        patch('apache.aurora.client.commands.core.make_v1_client', return_value=mock_api),
         patch('twitter.common.app.get_options', return_value=mock_options)) as (sleep, make_client,
         options):
       mock_query = self.create_mock_query()
@@ -234,7 +234,7 @@ class TestClientCreateCommand(AuroraClientCommandTest):
     mock_options = self.setup_mock_options()
     (mock_api, mock_scheduler_proxy) = self.create_mock_api()
     with contextlib.nested(
-        patch('apache.aurora.client.commands.core.make_client', return_value=mock_api),
+        patch('apache.aurora.client.commands.core.make_v1_client', return_value=mock_api),
         patch('twitter.common.app.get_options', return_value=mock_options)) as (make_client,
         options):
       with temporary_file() as fp:
@@ -258,7 +258,7 @@ class TestClientCreateCommand(AuroraClientCommandTest):
     mock_options = self.setup_mock_options()
     (mock_api, mock_scheduler_proxy) = self.create_mock_api()
     with contextlib.nested(
-        patch('apache.aurora.client.commands.core.make_client', return_value=mock_api),
+        patch('apache.aurora.client.commands.core.make_v1_client', return_value=mock_api),
         patch('twitter.common.app.get_options', return_value=mock_options)) as (make_client,
         options):
       with temporary_file() as fp:

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8c841060/src/test/python/apache/aurora/client/commands/test_diff.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/commands/test_diff.py b/src/test/python/apache/aurora/client/commands/test_diff.py
index b48b53d..8e346ed 100644
--- a/src/test/python/apache/aurora/client/commands/test_diff.py
+++ b/src/test/python/apache/aurora/client/commands/test_diff.py
@@ -125,6 +125,7 @@ class TestDiffCommand(AuroraClientCommandTest):
             options,
             subprocess_patch,
             json_patch):
+
       with temporary_file() as fp:
         fp.write(self.get_valid_config())
         fp.flush()
@@ -187,6 +188,7 @@ class TestDiffCommand(AuroraClientCommandTest):
             options,
             subprocess_patch,
             json_patch):
+
       with temporary_file() as fp:
         fp.write(self.get_valid_config())
         fp.flush()

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/8c841060/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 2045f64..49a4d69 100644
--- a/src/test/python/apache/aurora/common/test_transport.py
+++ b/src/test/python/apache/aurora/common/test_transport.py
@@ -23,7 +23,7 @@ from thrift.protocol import TJSONProtocol
 from thrift.server import THttpServer
 from thrift.transport import TTransport
 
-from apache.aurora.common.transport import TRequestsTransport
+from apache.aurora.common.transport import DEFAULT_USER_AGENT, TRequestsTransport
 
 from gen.apache.aurora.api import ReadOnlyScheduler
 from gen.apache.aurora.api.ttypes import Response, ResponseCode, ServerInfo
@@ -118,6 +118,19 @@ def test_requests_transports_lowers_logging_level():
 
   TRequestsTransport(
       'http://localhost:12345',
-      session_factory=lambda: create_autospec(spec=requests.Session, instance=True))
+      session_factory=lambda x: create_autospec(spec=requests.Session, instance=True))
 
   assert logging.getLogger('requests').level == logging.WARNING
+
+
+def test_transport_applies_user_agent_from_factory():
+  user_agent = 'Some-User-Agent'
+  transport = TRequestsTransport('http://localhost:12345', user_agent=user_agent)
+  transport.open()
+  assert transport._session.headers['User-Agent'] == user_agent
+
+
+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