You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by wf...@apache.org on 2014/12/05 03:11:39 UTC

incubator-aurora git commit: Remove Response.messageDEPRECATED field.

Repository: incubator-aurora
Updated Branches:
  refs/heads/master 1df1a26e1 -> 905137f83


Remove Response.messageDEPRECATED field.

Bugs closed: AURORA-466

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


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

Branch: refs/heads/master
Commit: 905137f8360abe38d9a89dd4d32d99d91c0d9147
Parents: 1df1a26
Author: Bill Farner <wf...@apache.org>
Authored: Thu Dec 4 18:11:00 2014 -0800
Committer: Bill Farner <wf...@apache.org>
Committed: Thu Dec 4 18:11:00 2014 -0800

----------------------------------------------------------------------
 .../thrift/org/apache/aurora/gen/api.thrift     |  2 -
 .../apache/aurora/scheduler/thrift/Util.java    |  5 --
 src/main/python/apache/aurora/client/api/BUILD  |  2 +
 .../python/apache/aurora/client/api/__init__.py |  3 +-
 .../apache/aurora/client/api/command_runner.py  |  8 +-
 .../apache/aurora/client/api/quota_check.py     | 14 +++-
 .../apache/aurora/client/api/restarter.py       |  4 +-
 .../python/apache/aurora/client/api/updater.py  | 12 +--
 src/main/python/apache/aurora/client/base.py    | 11 ++-
 .../python/apache/aurora/client/cli/context.py  |  4 +-
 .../python/apache/aurora/client/cli/task.py     |  3 +-
 .../apache/aurora/client/commands/admin.py      |  3 +-
 .../apache/aurora/client/commands/core.py       | 11 +--
 .../apache/aurora/client/hooks/hooked_api.py    |  3 +-
 .../thrift/SchedulerThriftInterfaceTest.java    | 10 +--
 .../python/apache/aurora/client/api/test_api.py |  3 +-
 .../aurora/client/api/test_disambiguator.py     | 11 ++-
 .../aurora/client/api/test_instance_watcher.py  | 12 ++-
 .../aurora/client/api/test_job_monitor.py       |  3 +-
 .../aurora/client/api/test_quota_check.py       |  3 +-
 .../apache/aurora/client/api/test_restarter.py  | 31 ++++----
 .../python/apache/aurora/client/api/test_sla.py |  3 +-
 .../apache/aurora/client/api/test_task_util.py  |  3 +-
 .../apache/aurora/client/api/test_updater.py    | 79 ++++++++------------
 .../python/apache/aurora/client/cli/util.py     |  6 +-
 .../apache/aurora/client/commands/test_admin.py | 28 +++----
 .../apache/aurora/client/commands/test_kill.py  |  3 +-
 .../apache/aurora/client/commands/util.py       |  5 +-
 .../python/apache/aurora/client/test_base.py    | 14 +++-
 29 files changed, 155 insertions(+), 144 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/905137f8/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 7d55dce..5665c69 100644
--- a/api/src/main/thrift/org/apache/aurora/gen/api.thrift
+++ b/api/src/main/thrift/org/apache/aurora/gen/api.thrift
@@ -878,8 +878,6 @@ struct ResponseDetail {
 
 struct Response {
   1: ResponseCode responseCode
-  // TODO(wfarner): Remove the message field in 0.7.0. (AURORA-466)
-  2: optional string messageDEPRECATED
   5: ServerInfo serverInfo
   /** Payload from the invoked RPC. */
   3: optional Result result

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/905137f8/src/main/java/org/apache/aurora/scheduler/thrift/Util.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/thrift/Util.java b/src/main/java/org/apache/aurora/scheduler/thrift/Util.java
index d879db4..55242d1 100644
--- a/src/main/java/org/apache/aurora/scheduler/thrift/Util.java
+++ b/src/main/java/org/apache/aurora/scheduler/thrift/Util.java
@@ -13,7 +13,6 @@
  */
 package org.apache.aurora.scheduler.thrift;
 
-import com.google.common.base.Strings;
 import com.google.common.collect.Lists;
 import com.twitter.common.base.MorePreconditions;
 
@@ -80,10 +79,6 @@ public final class Util {
   }
 
   private static Response appendMessage(Response response, String message) {
-    String existingMessage = response.getMessageDEPRECATED();
-    String prefix = Strings.isNullOrEmpty(existingMessage) ? "" : existingMessage + ", ";
-    response.setMessageDEPRECATED(prefix + message);
-
     response.addToDetails(new ResponseDetail(message));
     return response;
   }

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/905137f8/src/main/python/apache/aurora/client/api/BUILD
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/api/BUILD b/src/main/python/apache/aurora/client/api/BUILD
index 8b0da67..65e5a85 100644
--- a/src/main/python/apache/aurora/client/api/BUILD
+++ b/src/main/python/apache/aurora/client/api/BUILD
@@ -22,6 +22,7 @@ python_library(
     ':updater',
     '3rdparty/python:twitter.common.lang',
     '3rdparty/python:twitter.common.log',
+    'src/main/python/apache/aurora/client:base',
     'src/main/python/apache/aurora/common',
     'api/src/main/thrift/org/apache/aurora/gen:py-thrift',
   ]
@@ -139,6 +140,7 @@ python_library(
   dependencies = [
     ':scheduler_client',
     '3rdparty/python:twitter.common.log',
+    'src/main/python/apache/aurora/client:base',
     'api/src/main/thrift/org/apache/aurora/gen:py-thrift',
   ]
 )

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/905137f8/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 d12132f..9977b41 100644
--- a/src/main/python/apache/aurora/client/api/__init__.py
+++ b/src/main/python/apache/aurora/client/api/__init__.py
@@ -16,6 +16,7 @@ from __future__ import print_function
 
 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.cluster import Cluster
@@ -237,7 +238,7 @@ class AuroraClientAPI(object):
     log.info("Canceling update on job %s" % job_key)
     resp = Updater.cancel_update(self._scheduler_proxy, job_key)
     if resp.responseCode != ResponseCode.OK:
-      log.error('Error cancelling the update: %s' % resp.messageDEPRECATED)
+      log.error('Error cancelling the update: %s' % combine_messages(resp))
     return resp
 
   def restart(self, job_key, instances, updater_config, health_check_interval_seconds):

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/905137f8/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 14a316b..4d8f010 100644
--- a/src/main/python/apache/aurora/client/api/command_runner.py
+++ b/src/main/python/apache/aurora/client/api/command_runner.py
@@ -23,6 +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.common.cluster import Cluster
 from apache.aurora.config.schema.base import MesosContext
 from apache.thermos.config.schema import ThermosContext
@@ -158,6 +159,7 @@ class InstanceDistributedCommandRunner(DistributedCommandRunner):
       for task in resp.result.scheduleStatusResult.tasks:
         yield task
     else:
-      self._log(logging.ERROR,
-          "Error: could not retrieve task information for run command: %s" % resp.messageDEPRECATED)
-      raise ValueError("Could not retrieve task information: %s" % resp.messageDEPRECATED)
+      self._log(
+          logging.ERROR,
+          'Error: could not retrieve task information for run command: %s' % combine_messages(resp))
+      raise ValueError('Could not retrieve task information: %s' % combine_messages(resp))

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/905137f8/src/main/python/apache/aurora/client/api/quota_check.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/api/quota_check.py b/src/main/python/apache/aurora/client/api/quota_check.py
index 5877cba..c994050 100644
--- a/src/main/python/apache/aurora/client/api/quota_check.py
+++ b/src/main/python/apache/aurora/client/api/quota_check.py
@@ -17,7 +17,9 @@ from copy import deepcopy
 
 from twitter.common import log
 
-from gen.apache.aurora.api.ttypes import ResourceAggregate, Response, ResponseCode
+from apache.aurora.client.base import combine_messages
+
+from gen.apache.aurora.api.ttypes import ResourceAggregate, Response, ResponseCode, ResponseDetail
 
 
 class CapacityRequest(object):
@@ -75,13 +77,16 @@ class QuotaCheck(object):
 
     Returns: ResponseCode.OK if check is successful.
     """
-    resp_ok = Response(responseCode=ResponseCode.OK, messageDEPRECATED='Quota check successful.')
+    # TODO(wfarner): Avoid synthesizing scheduler responses.
+    resp_ok = Response(
+        responseCode=ResponseCode.OK,
+        details=[ResponseDetail(message='Quota check successful.')])
     if not production:
       return resp_ok
 
     resp = self._scheduler.getQuota(job_key.role)
     if resp.responseCode != ResponseCode.OK:
-      log.error('Failed to get quota from scheduler: %s' % resp.messageDEPRECATED)
+      log.error('Failed to get quota from scheduler: %s' % combine_messages(resp))
       return resp
 
     allocated = CapacityRequest(resp.result.getQuotaResult.quota)
@@ -94,9 +99,10 @@ class QuotaCheck(object):
       print_quota(allocated.quota(), 'Total allocated quota', job_key.role)
       print_quota(consumed.quota(), 'Consumed quota', job_key.role)
       print_quota(requested.quota(), 'Requested', job_key.name)
+      # TODO(wfarner): Avoid synthesizing scheduler responses.
       return Response(
           responseCode=ResponseCode.INVALID_REQUEST,
-          messageDEPRECATED='Failed quota check.')
+          details=[ResponseDetail(message='Failed quota check.')])
 
     return resp_ok
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/905137f8/src/main/python/apache/aurora/client/api/restarter.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/api/restarter.py b/src/main/python/apache/aurora/client/api/restarter.py
index 43599e7..1983034 100644
--- a/src/main/python/apache/aurora/client/api/restarter.py
+++ b/src/main/python/apache/aurora/client/api/restarter.py
@@ -14,6 +14,8 @@
 
 from twitter.common import log
 
+from apache.aurora.client.base import combine_messages
+
 from .instance_watcher import InstanceWatcher
 from .updater_util import FailureThreshold
 
@@ -72,7 +74,7 @@ class Restarter(object):
 
       resp = self._scheduler.restartShards(self._job_key.to_thrift(), batch, self._lock)
       if resp.responseCode != ResponseCode.OK:
-        log.error('Error restarting instances: %s', resp.messageDEPRECATED)
+        log.error('Error restarting instances: %s', combine_messages(resp))
         return resp
 
       failed_instances = self._instance_watcher.watch(batch)

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/905137f8/src/main/python/apache/aurora/client/api/updater.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/api/updater.py b/src/main/python/apache/aurora/client/api/updater.py
index 2092ff3..1c5708c 100644
--- a/src/main/python/apache/aurora/client/api/updater.py
+++ b/src/main/python/apache/aurora/client/api/updater.py
@@ -24,7 +24,7 @@ from thrift.TSerialization import serialize
 from twitter.common import log
 from twitter.common.quantity import Amount, Time
 
-from apache.aurora.client.base import format_response
+from apache.aurora.client.base import combine_messages, format_response
 
 from .error_handling_thread import ExecutionError, spawn_worker
 from .instance_watcher import InstanceWatcher
@@ -132,7 +132,7 @@ class Updater(object):
     if resp.responseCode == ResponseCode.OK:
       self._lock = None
     else:
-      log.error('There was an error finalizing the update: %s' % resp.messageDEPRECATED)
+      log.error('There was an error finalizing the update: %s' % combine_messages(resp))
     return resp
 
   def int_handler(self, *args):
@@ -646,12 +646,8 @@ class Updater(object):
     return TaskQuery(jobKeys=[self._job_key], statuses=ACTIVE_STATES, instanceIds=instanceIds)
 
   def _failed_response(self, message):
-    # This is a synthetic Scheduler response that should follow the dual-write approach for error
-    # messages to simplify caller's handling logic until the deprecation (AURORA-466) is complete.
-    return Response(
-        responseCode=ResponseCode.ERROR,
-        details=[ResponseDetail(message=message)],
-        messageDEPRECATED=message)
+    # TODO(wfarner): Avoid synthesizing scheduler responses, consider using an exception instead.
+    return Response(responseCode=ResponseCode.ERROR, details=[ResponseDetail(message=message)])
 
   def update(self, instances=None):
     """Performs the job update, blocking until it completes.

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/905137f8/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 2c7d816..a897e87 100644
--- a/src/main/python/apache/aurora/client/base.py
+++ b/src/main/python/apache/aurora/client/base.py
@@ -40,10 +40,17 @@ def die(msg):
   sys.exit(1)
 
 
+def combine_messages(response):
+  """Combines the message found in the details of a response.
+  :param response: response to extract messages from.
+  :return: Messages from the details in the response, or an empty string if there were no messages.
+  """
+  return ', '.join([d.message for d in (response.details or [])])
+
+
 def format_response(resp):
   return 'Response from scheduler: %s (message: %s)' % (
-    ResponseCode._VALUES_TO_NAMES[resp.responseCode],
-    ", ".join([m.message for m in resp.details]) if resp.details else resp.messageDEPRECATED)
+    ResponseCode._VALUES_TO_NAMES[resp.responseCode], combine_messages(resp))
 
 
 def check_and_log_response(resp):

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/905137f8/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 96c386e..5e89296 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 synthesize_url
+from apache.aurora.client.base import combine_messages, synthesize_url
 from apache.aurora.client.cli import (
     Context,
     EXIT_API_ERROR,
@@ -149,7 +149,7 @@ class AuroraCommandContext(Context):
   def check_and_log_response(self, resp, err_code=EXIT_API_ERROR, err_msg=None):
     if resp.responseCode != ResponseCode.OK:
       if err_msg is None:
-        err_msg = resp.messageDEPRECATED
+        err_msg = combine_messages(resp)
       if resp.responseCode == ResponseCode.LOCK_ERROR:
         self.print_err(self.LOCK_ERROR_MSG)
       self.print_err(err_msg)

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/905137f8/src/main/python/apache/aurora/client/cli/task.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/task.py b/src/main/python/apache/aurora/client/cli/task.py
index 8a139db..c92111e 100644
--- a/src/main/python/apache/aurora/client/cli/task.py
+++ b/src/main/python/apache/aurora/client/cli/task.py
@@ -23,6 +23,7 @@ from apache.aurora.client.api.command_runner import (
     DistributedCommandRunner,
     InstanceDistributedCommandRunner
 )
+from apache.aurora.client.base import combine_messages
 from apache.aurora.client.cli import EXIT_INVALID_PARAMETER, EXIT_OK, Noun, print_aurora_log, Verb
 from apache.aurora.client.cli.context import AuroraCommandContext
 from apache.aurora.client.cli.options import (
@@ -104,7 +105,7 @@ class SshCommand(Verb):
     api = context.get_api(cluster)
     resp = api.query(api.build_query(role, name, set([int(instance)]), env=env))
     context.check_and_log_response(resp,
-        err_msg=('Unable to get information about instance: %s' % resp.messageDEPRECATED))
+        err_msg=('Unable to get information about instance: %s' % combine_messages(resp)))
     if (resp.result.scheduleStatusResult.tasks is None or
         len(resp.result.scheduleStatusResult.tasks) == 0):
       raise context.CommandError(EXIT_INVALID_PARAMETER,

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/905137f8/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 cb5ae88..4b86c5a 100644
--- a/src/main/python/apache/aurora/client/commands/admin.py
+++ b/src/main/python/apache/aurora/client/commands/admin.py
@@ -36,6 +36,7 @@ from apache.aurora.client.api import AuroraClientAPI
 from apache.aurora.client.api.sla import JobUpTimeLimit
 from apache.aurora.client.base import (
     check_and_log_response,
+    combine_messages,
     die,
     get_grouping_or_die,
     GROUPING_OPTION,
@@ -144,7 +145,7 @@ def query(args, options):
   api = AuroraClientAPI(CLUSTERS[cluster], options.verbosity)
   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' % query_info.messageDEPRECATED)
+    die('Failed to query scheduler: %s' % combine_messages(query_info))
 
   tasks = query_info.result.scheduleStatusResult.tasks
   if tasks is None:

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/905137f8/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 ee22716..8d92862 100644
--- a/src/main/python/apache/aurora/client/commands/core.py
+++ b/src/main/python/apache/aurora/client/commands/core.py
@@ -36,6 +36,7 @@ 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 (
     check_and_log_response,
+    combine_messages,
     deprecation_warning,
     die,
     handle_open,
@@ -274,11 +275,11 @@ def diff(job_spec, config_file):
   api = make_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"' % resp.messageDEPRECATED)
+    die('Request failed, server responded with "%s"' % combine_messages(resp))
   remote_tasks = [t.assignedTask.task for t in resp.result.scheduleStatusResult.tasks]
   resp = api.populate_job_config(config)
   if resp.responseCode != ResponseCode.OK:
-    die('Request failed, server responded with "%s"' % resp.messageDEPRECATED)
+    die('Request failed, server responded with "%s"' % combine_messages(resp))
   local_tasks = resp.result.populateJobResult.populatedDEPRECATED
 
   pp = pprint.PrettyPrinter(indent=2)
@@ -572,7 +573,7 @@ def kill_in_batches(api, job_key, instances_arg, batch_size, max_failures):
     for batch in make_batches(instances_to_kill, batch_size):
       resp = api.kill_job(job_key, batch)
       if resp.responseCode is not ResponseCode.OK:
-        log.info("Kill of shards %s failed with error %s" % (batch, resp.messageDEPRECATED))
+        log.info("Kill of shards %s failed with error %s" % (batch, combine_messages(resp)))
         print('ERROR IN KILL_JOB')
         errors += 1
         if errors > max_failures:
@@ -703,11 +704,11 @@ def really_update(job_spec, config_file, options):
     resp = api.query_no_configs(api.build_query(config.role(), config.name(),
         statuses=ACTIVE_STATES, env=config.environment()))
     if resp.responseCode != ResponseCode.OK:
-      die('Could not get job status from server for comparison: %s' % resp.messageDEPRECATED)
+      die('Could not get job status from server for comparison: %s' % combine_messages(resp))
     remote_tasks = [t.assignedTask.task for t in resp.result.scheduleStatusResult.tasks]
     resp = api.populate_job_config(config)
     if resp.responseCode != ResponseCode.OK:
-      die('Server could not populate job config for comparison: %s' % resp.messageDEPRECATED)
+      die('Server could not populate job config for comparison: %s' % combine_messages(resp))
     local_task_count = len(resp.result.populateJobResult.populatedDEPRECATED)
     remote_task_count = len(remote_tasks)
     if (local_task_count >= 4 * remote_task_count or local_task_count <= 4 * remote_task_count

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/905137f8/src/main/python/apache/aurora/client/hooks/hooked_api.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/hooks/hooked_api.py b/src/main/python/apache/aurora/client/hooks/hooked_api.py
index 91efe52..bc61e91 100644
--- a/src/main/python/apache/aurora/client/hooks/hooked_api.py
+++ b/src/main/python/apache/aurora/client/hooks/hooked_api.py
@@ -18,6 +18,7 @@ import traceback
 from twitter.common import log
 
 from apache.aurora.client.api import AuroraClientAPI
+from apache.aurora.client.base import combine_messages
 from apache.aurora.client.config import GlobalHookRegistry
 
 from gen.apache.aurora.api.ttypes import ResponseCode
@@ -84,7 +85,7 @@ class HookedAuroraClientAPI(NonHookedAuroraClientAPI):
     def __str__(self):
       return '%s: %s: %s' % (self.__class__.__name__,
           ResponseCode._VALUES_TO_NAMES.get(self.response.responseCode, 'UNKNOWN'),
-          self.response.messageDEPRECATED)
+          combine_messages(self.response))
 
   @classmethod
   def _meta_hook(cls, hook, hook_method):

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/905137f8/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java b/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
index 072ea2b..8d41e70 100644
--- a/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
@@ -26,7 +26,6 @@ import java.util.UUID;
 import javax.annotation.Nullable;
 
 import com.google.common.base.Function;
-import com.google.common.base.Joiner;
 import com.google.common.base.Optional;
 import com.google.common.base.Predicate;
 import com.google.common.base.Strings;
@@ -542,7 +541,6 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
   private void assertMessageMatches(Response response, final String string) {
     // TODO(wfarner): This test coverage could be much better.  Circle back to apply more thorough
     // response contents testing throughout.
-    assertEquals(string, response.getMessageDEPRECATED());
     assertTrue(Iterables.any(response.getDetails(), new Predicate<ResponseDetail>() {
       @Override
       public boolean apply(ResponseDetail detail) {
@@ -1790,10 +1788,8 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
         .setServerInfo(SERVER_INFO)
         .setResult(result.orNull());
     if (messages.length > 0) {
-      response.setMessageDEPRECATED(Joiner.on(", ").join(messages));
-      response
-          .setDetails(FluentIterable.from(Arrays.asList(messages)).transform(MESSAGE_TO_DETAIL)
-              .toList());
+      response.setDetails(FluentIterable.from(Arrays.asList(messages)).transform(MESSAGE_TO_DETAIL)
+          .toList());
     }
 
     return response;
@@ -1809,7 +1805,6 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
 
   private Response errorResponse(String message) {
     return response(ERROR, Optional.<Result>absent())
-        .setMessageDEPRECATED(message)
         .setDetails(ImmutableList.of(new ResponseDetail().setMessage(message)));
   }
 
@@ -1817,7 +1812,6 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
     return Util.emptyResponse()
         .setResponseCode(INVALID_REQUEST)
         .setServerInfo(SERVER_INFO)
-        .setMessageDEPRECATED(message)
         .setDetails(ImmutableList.of(new ResponseDetail(message)));
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/905137f8/src/test/python/apache/aurora/client/api/test_api.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/api/test_api.py b/src/test/python/apache/aurora/client/api/test_api.py
index 1f4e9fe..c91e74e 100644
--- a/src/test/python/apache/aurora/client/api/test_api.py
+++ b/src/test/python/apache/aurora/client/api/test_api.py
@@ -31,6 +31,7 @@ from gen.apache.aurora.api.ttypes import (
     JobUpdateStatus,
     Response,
     ResponseCode,
+    ResponseDetail,
     Result,
     TaskConfig
 )
@@ -55,7 +56,7 @@ class TestJobUpdateApis(unittest.TestCase):
   def create_blank_response(cls, code, msg):
     return Response(
       responseCode=code,
-      messageDEPRECATED=msg,
+      details=[ResponseDetail(message=msg)],
       result=create_autospec(spec=Result, spec_set=True, instance=True)
     )
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/905137f8/src/test/python/apache/aurora/client/api/test_disambiguator.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/api/test_disambiguator.py b/src/test/python/apache/aurora/client/api/test_disambiguator.py
index e9523ac..72e98d2 100644
--- a/src/test/python/apache/aurora/client/api/test_disambiguator.py
+++ b/src/test/python/apache/aurora/client/api/test_disambiguator.py
@@ -23,7 +23,14 @@ from apache.aurora.common.aurora_job_key import AuroraJobKey
 from apache.aurora.common.cluster import Cluster
 
 from gen.apache.aurora.api.constants import ResponseCode
-from gen.apache.aurora.api.ttypes import GetJobsResult, JobConfiguration, JobKey, Response, Result
+from gen.apache.aurora.api.ttypes import (
+    GetJobsResult,
+    JobConfiguration,
+    JobKey,
+    Response,
+    ResponseDetail,
+    Result
+)
 
 TEST_CLUSTER = Cluster(name='smf1')
 
@@ -50,7 +57,7 @@ class LiveJobDisambiguatorTest(mox.MoxTestBase):
   def _expect_get_jobs(self, *envs):
     self._api.get_jobs(self.ROLE).AndReturn(Response(
       responseCode=ResponseCode.OK,
-      messageDEPRECATED='Mock OK',
+      details=[ResponseDetail(message='Mock OK')],
       result=Result(getJobsResult=GetJobsResult(
         configs=set(JobConfiguration(key=JobKey(role=self.ROLE, environment=env, name=self.NAME))
         for env in envs)))))

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/905137f8/src/test/python/apache/aurora/client/api/test_instance_watcher.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/api/test_instance_watcher.py b/src/test/python/apache/aurora/client/api/test_instance_watcher.py
index abbbdbe..9efe1d4 100644
--- a/src/test/python/apache/aurora/client/api/test_instance_watcher.py
+++ b/src/test/python/apache/aurora/client/api/test_instance_watcher.py
@@ -26,6 +26,7 @@ from gen.apache.aurora.api.ttypes import (
     JobKey,
     Response,
     ResponseCode,
+    ResponseDetail,
     Result,
     ScheduledTask,
     ScheduleStatus,
@@ -97,9 +98,10 @@ class InstanceWatcherTest(unittest.TestCase):
 
   def expect_get_statuses(self, instance_ids=WATCH_INSTANCES, num_calls=EXPECTED_CYCLES):
     tasks = [self.create_task(instance_id) for instance_id in instance_ids]
-    response = Response(responseCode=ResponseCode.OK, messageDEPRECATED='test')
-    response.result = Result()
-    response.result.scheduleStatusResult = ScheduleStatusResult(tasks=tasks)
+    response = Response(
+        responseCode=ResponseCode.OK,
+        details=[ResponseDetail(message='test')],
+        result=Result(scheduleStatusResult=ScheduleStatusResult(tasks=tasks)))
 
     query = self.get_tasks_status_query(instance_ids)
     for _ in range(int(num_calls)):
@@ -107,10 +109,6 @@ class InstanceWatcherTest(unittest.TestCase):
 
   def expect_io_error_in_get_statuses(self, instance_ids=WATCH_INSTANCES,
       num_calls=EXPECTED_CYCLES):
-    tasks = [self.create_task(instance_id) for instance_id in instance_ids]
-    response = Response(responseCode=ResponseCode.OK, messageDEPRECATED='test')
-    response.result = Result()
-    response.result.scheduleStatusResult = ScheduleStatusResult(tasks=tasks)
 
     query = self.get_tasks_status_query(instance_ids)
     for _ in range(int(num_calls)):

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/905137f8/src/test/python/apache/aurora/client/api/test_job_monitor.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/api/test_job_monitor.py b/src/test/python/apache/aurora/client/api/test_job_monitor.py
index 27d8025..1e7a879 100644
--- a/src/test/python/apache/aurora/client/api/test_job_monitor.py
+++ b/src/test/python/apache/aurora/client/api/test_job_monitor.py
@@ -25,6 +25,7 @@ from gen.apache.aurora.api.ttypes import (
     JobKey,
     Response,
     ResponseCode,
+    ResponseDetail,
     Result,
     ScheduledTask,
     ScheduleStatus,
@@ -68,7 +69,7 @@ class JobMonitorTest(unittest.TestCase):
 
   def mock_get_tasks(self, tasks, response_code=None):
     response_code = ResponseCode.OK if response_code is None else response_code
-    resp = Response(responseCode=response_code, messageDEPRECATED='test')
+    resp = Response(responseCode=response_code, details=[ResponseDetail(message='test')])
     resp.result = Result(scheduleStatusResult=ScheduleStatusResult(tasks=tasks))
     self._scheduler.getTasksWithoutConfigs.return_value = resp
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/905137f8/src/test/python/apache/aurora/client/api/test_quota_check.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/api/test_quota_check.py b/src/test/python/apache/aurora/client/api/test_quota_check.py
index cb443c2..2fc76d2 100644
--- a/src/test/python/apache/aurora/client/api/test_quota_check.py
+++ b/src/test/python/apache/aurora/client/api/test_quota_check.py
@@ -27,6 +27,7 @@ from gen.apache.aurora.api.ttypes import (
     ResourceAggregate,
     Response,
     ResponseCode,
+    ResponseDetail,
     Result
 )
 
@@ -43,7 +44,7 @@ class QuotaCheckTest(unittest.TestCase):
   def mock_get_quota(self, allocated, consumed, response_code=None):
     response_code = ResponseCode.OK if response_code is None else response_code
 
-    resp = Response(responseCode=response_code, messageDEPRECATED='test')
+    resp = Response(responseCode=response_code, details=[ResponseDetail(message='test')])
     resp.result = Result(
         getQuotaResult=GetQuotaResult(
           quota=deepcopy(allocated), prodConsumption=deepcopy(consumed)))

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/905137f8/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 eb0af3b..3b14d88 100644
--- a/src/test/python/apache/aurora/client/api/test_restarter.py
+++ b/src/test/python/apache/aurora/client/api/test_restarter.py
@@ -28,6 +28,7 @@ from gen.apache.aurora.api.ttypes import (
     AssignedTask,
     Response,
     ResponseCode,
+    ResponseDetail,
     Result,
     ScheduledTask,
     ScheduleStatus,
@@ -50,6 +51,14 @@ UPDATER_CONFIG = UpdaterConfig(
 )
 
 
+def make_response(code=ResponseCode.OK, message='test', result=None):
+  return Response(
+    responseCode=code,
+    details=[ResponseDetail(message=message)],
+    result=result,
+    serverInfo=ServerInfo(clusterName='test', thriftAPIVersion=THRIFT_API_VERSION))
+
+
 class TestRestarter(MoxTestBase):
 
   def setUp(self):
@@ -70,14 +79,11 @@ class TestRestarter(MoxTestBase):
         self.mock_instance_watcher)
 
   def mock_restart_instances(self, instances, lock=None):
-    response = Response(responseCode=ResponseCode.OK, messageDEPRECATED='test')
-    response.serverInfo = ServerInfo(thriftAPIVersion=THRIFT_API_VERSION)
-
     self.mock_scheduler.restartShards(
         JOB.to_thrift(),
         instances,
         lock,
-        SESSION_KEY).AndReturn(response)
+        SESSION_KEY).AndReturn(make_response())
     self.mock_instance_watcher.watch(instances).AndReturn([])
 
   def test_restart_one_iteration(self):
@@ -108,11 +114,7 @@ class TestRestarter(MoxTestBase):
           status=ScheduleStatus.RUNNING,
           assignedTask=AssignedTask(task=TaskConfig(), instanceId=i)
       ))
-    response = Response(responseCode=ResponseCode.OK, messageDEPRECATED='test')
-    response.serverInfo = ServerInfo(thriftAPIVersion=THRIFT_API_VERSION)
-    response.result = Result()
-    response.result.scheduleStatusResult = ScheduleStatusResult(tasks=tasks)
-
+    response = make_response(result=Result(scheduleStatusResult=ScheduleStatusResult(tasks=tasks)))
     self.mock_scheduler.getTasksWithoutConfigs(IgnoreArg()).AndReturn(response)
 
   def test_restart_all_instances(self):
@@ -124,8 +126,7 @@ class TestRestarter(MoxTestBase):
     self.restarter.restart(None)
 
   def mock_status_no_active_task(self):
-    response = Response(responseCode=ResponseCode.INVALID_REQUEST, messageDEPRECATED='test')
-    response.serverInfo = ServerInfo(thriftAPIVersion=THRIFT_API_VERSION)
+    response = make_response(code=ResponseCode.INVALID_REQUEST)
     self.mock_scheduler.getTasksWithoutConfigs(IgnoreArg()).AndReturn(response)
 
   def test_restart_no_instance_active(self):
@@ -136,8 +137,7 @@ class TestRestarter(MoxTestBase):
     self.restarter.restart(None)
 
   def mock_restart_fails(self):
-    response = Response(responseCode=ResponseCode.ERROR, messageDEPRECATED='test error')
-    response.serverInfo = ServerInfo(thriftAPIVersion=THRIFT_API_VERSION)
+    response = make_response(code=ResponseCode.ERROR, message='test error')
     self.mock_scheduler.restartShards(
         JOB.to_thrift(),
         IgnoreArg(),
@@ -153,14 +153,11 @@ class TestRestarter(MoxTestBase):
     assert self.restarter.restart(None).responseCode == ResponseCode.ERROR
 
   def mock_restart_watch_fails(self, instances):
-    response = Response(responseCode=ResponseCode.OK, messageDEPRECATED='test')
-    response.serverInfo = ServerInfo(thriftAPIVersion=THRIFT_API_VERSION)
-
     self.mock_scheduler.restartShards(
         JOB.to_thrift(),
         instances,
         self.lock,
-        SESSION_KEY).AndReturn(response)
+        SESSION_KEY).AndReturn(make_response())
     self.mock_instance_watcher.watch(instances).AndReturn(instances)
 
   def test_restart_instances_watch_fails(self):

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/905137f8/src/test/python/apache/aurora/client/api/test_sla.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/api/test_sla.py b/src/test/python/apache/aurora/client/api/test_sla.py
index 50a6c47..22b9f99 100644
--- a/src/test/python/apache/aurora/client/api/test_sla.py
+++ b/src/test/python/apache/aurora/client/api/test_sla.py
@@ -30,6 +30,7 @@ from gen.apache.aurora.api.ttypes import (
     JobKey,
     Response,
     ResponseCode,
+    ResponseDetail,
     Result,
     ScheduledTask,
     ScheduleStatus,
@@ -57,7 +58,7 @@ class SlaTest(unittest.TestCase):
 
   def mock_get_tasks(self, tasks, response_code=None):
     response_code = ResponseCode.OK if response_code is None else response_code
-    resp = Response(responseCode=response_code, messageDEPRECATED='test')
+    resp = Response(responseCode=response_code, details=[ResponseDetail(message='test')])
     resp.result = Result(scheduleStatusResult=ScheduleStatusResult(tasks=tasks))
     self._scheduler.getTasksWithoutConfigs.return_value = resp
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/905137f8/src/test/python/apache/aurora/client/api/test_task_util.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/api/test_task_util.py b/src/test/python/apache/aurora/client/api/test_task_util.py
index 3e772b9..048aff6 100644
--- a/src/test/python/apache/aurora/client/api/test_task_util.py
+++ b/src/test/python/apache/aurora/client/api/test_task_util.py
@@ -25,6 +25,7 @@ from gen.apache.aurora.api.ttypes import (
     AssignedTask,
     Response,
     ResponseCode,
+    ResponseDetail,
     Result,
     ScheduledTask,
     ScheduleStatusResult,
@@ -59,7 +60,7 @@ class TaskUtilTest(unittest.TestCase):
   def mock_scheduler(cls, response_code=None):
     scheduler = create_autospec(spec=SchedulerThriftApiSpec, instance=True)
     response_code = ResponseCode.OK if response_code is None else response_code
-    resp = Response(responseCode=response_code, messageDEPRECATED='test')
+    resp = Response(responseCode=response_code, details=[ResponseDetail(message='test')])
     resp.result = Result(scheduleStatusResult=ScheduleStatusResult(tasks=cls.create_tasks()))
     scheduler.getTasksWithoutConfigs.return_value = resp
     return scheduler

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/905137f8/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 a32fc52..dad69e2 100644
--- a/src/test/python/apache/aurora/client/api/test_updater.py
+++ b/src/test/python/apache/aurora/client/api/test_updater.py
@@ -48,6 +48,7 @@ from gen.apache.aurora.api.ttypes import (
     ResourceAggregate,
     Response,
     ResponseCode,
+    ResponseDetail,
     Result,
     ScheduledTask,
     ScheduleStatusResult,
@@ -69,6 +70,13 @@ if 'UPDATER_DEBUG' in environ:
 SERVER_INFO = ServerInfo(thriftAPIVersion=THRIFT_API_VERSION)
 
 
+def make_response(code, msg='test'):
+  return Response(
+      responseCode=code,
+      serverInfo=SERVER_INFO,
+      details=[ResponseDetail(message=msg)])
+
+
 class FakeConfig(object):
   def __init__(self, role, name, env, update_config):
     self._role = role
@@ -187,56 +195,49 @@ class UpdaterTest(TestCase):
       failed = [i] if i in failed_instances else []
       self._instance_watcher.watch(instance_ids).InAnyOrder().AndReturn(set(failed))
 
-  def expect_populate(self, job_config, response_code=None):
-    response_code = ResponseCode.OK if response_code is None else response_code
-    resp = Response(responseCode=response_code, messageDEPRECATED='test', serverInfo=SERVER_INFO)
+  def expect_populate(self, job_config, response_code=ResponseCode.OK):
+    resp = make_response(response_code)
     result = set([deepcopy(job_config.taskConfig)])
     resp.result = Result(populateJobResult=PopulateJobResult(populatedDEPRECATED=result))
     self._scheduler.populateJobConfig(job_config).AndReturn(resp)
 
-  def expect_get_tasks(self, tasks, ignore_ids=None, response_code=None):
-    response_code = ResponseCode.OK if response_code is None else response_code
-    response = Response(responseCode=response_code,
-                        messageDEPRECATED='test',
-                        serverInfo=SERVER_INFO)
+  def expect_get_tasks(self, tasks, ignore_ids=None, response_code=ResponseCode.OK):
     scheduled = []
     for index, task in enumerate(tasks):
       if not ignore_ids or index not in ignore_ids:
         scheduled.append(ScheduledTask(assignedTask=AssignedTask(task=task, instanceId=index)))
+    response = make_response(response_code)
     response.result = Result(scheduleStatusResult=ScheduleStatusResult(tasks=scheduled))
     query = TaskQuery(jobKeys=[self._job_key], statuses=ACTIVE_STATES)
     self._scheduler.getTasksStatus(query).AndReturn(response)
 
-  def expect_cron_replace(self, job_config, response_code=None):
-    response_code = ResponseCode.OK if response_code is None else response_code
-    resp = Response(responseCode=response_code, messageDEPRECATED='test', serverInfo=SERVER_INFO)
+  def expect_cron_replace(self, job_config, response_code=ResponseCode.OK):
+    resp = make_response(response_code)
     self._scheduler.replaceCronTemplate(job_config, self._lock, self._session_key).AndReturn(resp)
 
   def expect_restart(self, instance_ids, response_code=None):
     for i in instance_ids:
       response_code = ResponseCode.OK if response_code is None else response_code
-      response = Response(responseCode=response_code,
-                          messageDEPRECATED='test',
-                          serverInfo=SERVER_INFO)
+      response = make_response(response_code)
       self._scheduler.restartShards(
           self._job_key,
           [i],
           self._lock,
           self._session_key).AndReturn(response)
 
-  def expect_kill(self, instance_ids, response_code=None, monitor_result=True, skip_monitor=False):
+  def expect_kill(self,
+      instance_ids,
+      response_code=ResponseCode.OK,
+      monitor_result=True,
+      skip_monitor=False):
     for i in instance_ids:
-      response_code = ResponseCode.OK if response_code is None else response_code
-      response = Response(responseCode=response_code,
-                          messageDEPRECATED='test',
-                          serverInfo=SERVER_INFO)
       query = TaskQuery(jobKeys=[self._job_key],
                         statuses=ACTIVE_STATES,
                         instanceIds=frozenset([int(i)]))
       self._scheduler.killTasks(
           query,
           self._lock,
-          self._session_key).InAnyOrder().AndReturn(response)
+          self._session_key).InAnyOrder().AndReturn(make_response(response_code))
 
     self.expect_job_monitor(response_code, instance_ids, monitor_result, skip_monitor)
 
@@ -249,12 +250,8 @@ class UpdaterTest(TestCase):
         instance_ids,
         with_timeout=True).InAnyOrder().AndReturn(monitor_result)
 
-  def expect_add(self, instance_ids, task_config, response_code=None):
+  def expect_add(self, instance_ids, task_config, response_code=ResponseCode.OK):
     for i in instance_ids:
-      response_code = ResponseCode.OK if response_code is None else response_code
-      response = Response(responseCode=response_code,
-                          messageDEPRECATED='test',
-                          serverInfo=SERVER_INFO)
       add_config = AddInstancesConfig(
           key=self._job_key,
           taskConfig=task_config,
@@ -262,7 +259,7 @@ class UpdaterTest(TestCase):
       self._scheduler.addInstances(
           add_config,
           self._lock,
-          self._session_key).InAnyOrder().AndReturn(response)
+          self._session_key).InAnyOrder().AndReturn(make_response(response_code))
 
   def expect_update_instances(self, instance_ids, task_config):
     for i in instance_ids:
@@ -279,29 +276,22 @@ class UpdaterTest(TestCase):
     for i in instance_ids:
       self.expect_kill([i])
 
-  def expect_start(self, response_code=None):
-    response_code = ResponseCode.OK if response_code is None else response_code
-    response = Response(responseCode=response_code,
-                        messageDEPRECATED='test',
-                        serverInfo=SERVER_INFO)
+  def expect_start(self, response_code=ResponseCode.OK):
+    response = make_response(response_code)
     response.result = Result(acquireLockResult=AcquireLockResult(lock=self._lock))
     self._scheduler.acquireLock(LockKey(job=self._job_key), self._session_key).AndReturn(response)
 
-  def expect_finish(self, response_code=None):
-    response_code = ResponseCode.OK if response_code is None else response_code
-    response = Response(responseCode=response_code,
-                        messageDEPRECATED='test',
-                        serverInfo=SERVER_INFO)
+  def expect_finish(self, response_code=ResponseCode.OK):
     self._scheduler.releaseLock(
         self._lock,
         LockValidation.CHECKED,
-        self._session_key).AndReturn(response)
+        self._session_key).AndReturn(make_response(response_code))
 
-  def expect_quota_check(self, num_released, num_acquired, response_code=None, prod=True):
-    response_code = ResponseCode.OK if response_code is None else response_code
-    response = Response(responseCode=response_code,
-                        messageDEPRECATED='test',
-                        serverInfo=SERVER_INFO)
+  def expect_quota_check(self,
+      num_released,
+      num_acquired,
+      response_code=ResponseCode.OK,
+      prod=True):
     released = CapacityRequest(ResourceAggregate(
         numCpus=num_released * self._num_cpus,
         ramMb=num_released * self._num_ram,
@@ -312,7 +302,7 @@ class UpdaterTest(TestCase):
       diskMb=num_acquired * self._num_disk))
 
     self._quota_check.validate_quota_from_requested(
-        self._job_key, prod, released, acquired).AndReturn(response)
+        self._job_key, prod, released, acquired).AndReturn(make_response(response_code))
 
   def make_task_configs(self, count=1, prod=True):
     return [TaskConfig(
@@ -355,9 +345,6 @@ class UpdaterTest(TestCase):
       assert message in resp.details[0].message, (
         "Expected %s message not found in: %s" % (message, resp.details[0].message))
 
-      assert message in resp.messageDEPRECATED, (
-        "Expected %s message not found in: %s" % (message, resp.messageDEPRECATED))
-
   def test_grow(self):
     """Adds instances to the existing job."""
     old_configs = self.make_task_configs(3)

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/905137f8/src/test/python/apache/aurora/client/cli/util.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/util.py b/src/test/python/apache/aurora/client/cli/util.py
index 0ec74e6..67d7eaa 100644
--- a/src/test/python/apache/aurora/client/cli/util.py
+++ b/src/test/python/apache/aurora/client/cli/util.py
@@ -119,11 +119,7 @@ class AuroraClientCommandTest(unittest.TestCase):
 
   @classmethod
   def create_blank_response(cls, code, msg):
-    return Response(
-        responseCode=code,
-        details=[ResponseDetail(message=msg)],
-        messageDEPRECATED=msg
-    )
+    return Response(responseCode=code, details=[ResponseDetail(message=msg)])
 
   @classmethod
   def create_simple_success_response(cls):

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/905137f8/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 f9261af..c7d970c8 100644
--- a/src/test/python/apache/aurora/client/commands/test_admin.py
+++ b/src/test/python/apache/aurora/client/commands/test_admin.py
@@ -39,6 +39,7 @@ from gen.apache.aurora.api.ttypes import (
     ResourceAggregate,
     Response,
     ResponseCode,
+    ResponseDetail,
     Result,
     ScheduledTask,
     ScheduleStatus,
@@ -64,11 +65,11 @@ class TestQueryCommand(AuroraClientCommandTest):
     return mock_options
 
   @classmethod
-  def create_response(cls, tasks, response_code=None):
-    response_code = ResponseCode.OK if response_code is None else response_code
-    resp = Response(responseCode=response_code, messageDEPRECATED='test')
-    resp.result = Result(scheduleStatusResult=ScheduleStatusResult(tasks=tasks))
-    return resp
+  def create_response(cls, tasks, response_code=ResponseCode.OK):
+    return Response(
+      responseCode=response_code,
+      details=[ResponseDetail(message='test')],
+      result=Result(scheduleStatusResult=ScheduleStatusResult(tasks=tasks)))
 
   @classmethod
   def create_task(cls):
@@ -126,12 +127,13 @@ class TestQueryCommand(AuroraClientCommandTest):
 class TestIncreaseQuotaCommand(AuroraClientCommandTest):
 
   @classmethod
-  def create_response(cls, quota, prod, non_prod, response_code=None):
-    response_code = ResponseCode.OK if response_code is None else response_code
-    resp = Response(responseCode=response_code, messageDEPRECATED='test')
-    resp.result = Result(getQuotaResult=GetQuotaResult(
-        quota=quota, prodConsumption=prod, nonProdConsumption=non_prod))
-    return resp
+  def create_response(cls, quota, prod, non_prod, response_code=ResponseCode.OK):
+    return Response(
+        responseCode=response_code,
+        details=[ResponseDetail(message='test')],
+        result=Result(getQuotaResult=GetQuotaResult(
+            quota=quota, prodConsumption=prod, nonProdConsumption=non_prod))
+    )
 
   def test_increase_quota(self):
     """Tests successful execution of the increase_quota command."""
@@ -164,7 +166,7 @@ class TestSetQuotaCommand(AuroraClientCommandTest):
   @classmethod
   def create_response(cls, quota, prod, non_prod, response_code=None):
     response_code = ResponseCode.OK if response_code is None else response_code
-    resp = Response(responseCode=response_code, messageDEPRECATED='test')
+    resp = Response(responseCode=response_code, details=[ResponseDetail(message='test')])
     resp.result = Result(getQuotaResult=GetQuotaResult(
       quota=quota, prodConsumption=prod, nonProdConsumption=non_prod))
     return resp
@@ -204,7 +206,7 @@ class TestGetLocksCommand(AuroraClientCommandTest):
   @classmethod
   def create_response(cls, locks, response_code=None):
     response_code = ResponseCode.OK if response_code is None else response_code
-    resp = Response(responseCode=response_code, messageDEPRECATED='test')
+    resp = Response(responseCode=response_code, details=[ResponseDetail(message='test')])
     resp.result = Result(getLocksResult=GetLocksResult(locks=locks))
     return resp
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/905137f8/src/test/python/apache/aurora/client/commands/test_kill.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/commands/test_kill.py b/src/test/python/apache/aurora/client/commands/test_kill.py
index 4ac742f..52732ea 100644
--- a/src/test/python/apache/aurora/client/commands/test_kill.py
+++ b/src/test/python/apache/aurora/client/commands/test_kill.py
@@ -28,6 +28,7 @@ from gen.apache.aurora.api.ttypes import (
     JobKey,
     Response,
     ResponseCode,
+    ResponseDetail,
     Result,
     ScheduledTask,
     ScheduleStatus,
@@ -289,7 +290,7 @@ class TestClientKillCommand(AuroraClientCommandTest):
       ))
     return Response(
         responseCode=ResponseCode.OK,
-        messageDEPRECATED='Ok',
+        details=[ResponseDetail(message='Ok')],
         result=Result(scheduleStatusResult=ScheduleStatusResult(tasks=tasks))
     )
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/905137f8/src/test/python/apache/aurora/client/commands/util.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/commands/util.py b/src/test/python/apache/aurora/client/commands/util.py
index c06de50..0d93e76 100644
--- a/src/test/python/apache/aurora/client/commands/util.py
+++ b/src/test/python/apache/aurora/client/commands/util.py
@@ -25,7 +25,7 @@ from apache.aurora.common.clusters import Clusters
 
 from ..api.api_util import SchedulerProxyApiSpec
 
-from gen.apache.aurora.api.ttypes import Response, ResponseCode, Result
+from gen.apache.aurora.api.ttypes import Response, ResponseCode, ResponseDetail, Result
 
 
 class AuroraClientCommandTest(unittest.TestCase):
@@ -34,9 +34,8 @@ class AuroraClientCommandTest(unittest.TestCase):
     # TODO(wfarner): Don't use a mock here.
     response = create_autospec(spec=Response, instance=True)
     response.responseCode = code
-    response.messageDEPRECATED = msg
     response.result = create_autospec(spec=Result, instance=True)
-    response.details = []
+    response.details = [ResponseDetail(message=msg)]
     return response
 
   @classmethod

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/905137f8/src/test/python/apache/aurora/client/test_base.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/test_base.py b/src/test/python/apache/aurora/client/test_base.py
index 785784b..fba08ae 100644
--- a/src/test/python/apache/aurora/client/test_base.py
+++ b/src/test/python/apache/aurora/client/test_base.py
@@ -21,7 +21,7 @@ from gen.apache.aurora.api.ttypes import Response, ResponseCode, ResponseDetail
 class TestBase(unittest.TestCase):
 
   def test_format_response_with_message(self):
-    resp = Response(responseCode=ResponseCode.ERROR, messageDEPRECATED='Error')
+    resp = Response(responseCode=ResponseCode.ERROR, details=[ResponseDetail(message='Error')])
     formatted = base.format_response(resp)
     assert formatted == 'Response from scheduler: ERROR (message: Error)'
 
@@ -29,3 +29,15 @@ class TestBase(unittest.TestCase):
     resp = Response(responseCode=ResponseCode.ERROR, details=[ResponseDetail(message='Error')])
     formatted = base.format_response(resp)
     assert formatted == 'Response from scheduler: ERROR (message: Error)'
+
+  def test_combine_messages(self):
+    resp = Response(responseCode=ResponseCode.ERROR)
+    assert base.combine_messages(resp) == ''
+    resp = Response(responseCode=ResponseCode.ERROR, details=[])
+    assert base.combine_messages(resp) == ''
+    resp = Response(responseCode=ResponseCode.ERROR, details=[ResponseDetail(message='Error')])
+    assert base.combine_messages(resp) == 'Error'
+    resp = Response(
+        responseCode=ResponseCode.ERROR,
+        details=[ResponseDetail(message='Error1'), ResponseDetail(message='Error2')])
+    assert base.combine_messages(resp) == 'Error1, Error2'