You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by ma...@apache.org on 2015/01/07 21:56:27 UTC

incubator-aurora git commit: Implementing dual read the PopulatedJobConfig struct

Repository: incubator-aurora
Updated Branches:
  refs/heads/master dd2c8e067 -> 66a44b07c


Implementing dual read the PopulatedJobConfig struct

Bugs closed: AURORA-691

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


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

Branch: refs/heads/master
Commit: 66a44b07cd77e112ae8e05583f2ce08fdf9fc391
Parents: dd2c8e0
Author: Maxim Khutornenko <ma...@apache.org>
Authored: Wed Jan 7 12:54:46 2015 -0800
Committer: -l <ma...@apache.org>
Committed: Wed Jan 7 12:54:46 2015 -0800

----------------------------------------------------------------------
 .../python/apache/aurora/client/api/updater.py  |  6 ++----
 src/main/python/apache/aurora/client/base.py    |  9 +++++++++
 .../python/apache/aurora/client/cli/jobs.py     | 10 +++++-----
 .../apache/aurora/client/commands/core.py       | 13 ++++++------
 .../apache/aurora/client/api/test_updater.py    |  8 ++++++--
 .../apache/aurora/client/cli/test_diff.py       |  3 ++-
 .../apache/aurora/client/cli/test_restart.py    |  3 ++-
 .../apache/aurora/client/cli/test_update.py     |  3 ++-
 .../apache/aurora/client/commands/test_diff.py  |  3 ++-
 .../aurora/client/commands/test_restart.py      |  1 +
 .../aurora/client/commands/test_update.py       |  3 ++-
 .../python/apache/aurora/client/test_base.py    | 21 +++++++++++++++++++-
 12 files changed, 59 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/66a44b07/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 1c5708c..9f91de6 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 combine_messages, format_response
+from apache.aurora.client.base import combine_messages, format_response, get_populated_task_config
 
 from .error_handling_thread import ExecutionError, spawn_worker
 from .instance_watcher import InstanceWatcher
@@ -626,9 +626,7 @@ class Updater(object):
     """
     resp = self._scheduler.populateJobConfig(self._config.job())
     self._check_and_log_response(resp)
-
-    # Safe to take the first element as Scheduler would throw in case zero instances provided.
-    return list(resp.result.populateJobResult.populatedDEPRECATED)[0]
+    return get_populated_task_config(resp)
 
   def _replace_template_if_cron(self):
     """Checks if the provided job config represents a cron job and if so, replaces it.

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/66a44b07/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 258c1fa..0f7436a 100644
--- a/src/main/python/apache/aurora/client/base.py
+++ b/src/main/python/apache/aurora/client/base.py
@@ -68,6 +68,15 @@ def check_and_log_locked_response(resp):
     log.info(LOCKED_WARNING)
 
 
+def get_populated_task_config(resp):
+  """Temporary dual read until AURORA-975 is addressed."""
+  result = resp.result.populateJobResult
+  if result.taskConfig:
+    return result.taskConfig
+  else:
+    return list(result.populatedDEPRECATED)[0] if result.populatedDEPRECATED else None
+
+
 def deprecation_warning(text):
   log.warning('')
   log.warning('*' * 80)

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/66a44b07/src/main/python/apache/aurora/client/cli/jobs.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/jobs.py b/src/main/python/apache/aurora/client/cli/jobs.py
index 8bdb7f4..744d534 100644
--- a/src/main/python/apache/aurora/client/cli/jobs.py
+++ b/src/main/python/apache/aurora/client/cli/jobs.py
@@ -20,6 +20,7 @@ import os
 import pprint
 import subprocess
 import textwrap
+from copy import deepcopy
 from datetime import datetime
 from tempfile import NamedTemporaryFile
 
@@ -28,6 +29,7 @@ from thrift.TSerialization import serialize
 
 from apache.aurora.client.api.job_monitor import JobMonitor
 from apache.aurora.client.api.updater_util import UpdaterConfig
+from apache.aurora.client.base import get_populated_task_config
 from apache.aurora.client.cli import (
     EXIT_COMMAND_FAILURE,
     EXIT_INVALID_CONFIGURATION,
@@ -207,7 +209,8 @@ class DiffCommand(Verb):
     resp = api.populate_job_config(config)
     context.check_and_log_response(resp, err_code=EXIT_INVALID_CONFIGURATION,
           err_msg="Error loading configuration")
-    local_tasks = resp.result.populateJobResult.populatedDEPRECATED
+    # Deepcopy is important here as tasks will be modified for printing.
+    local_tasks = [deepcopy(get_populated_task_config(resp)) for _ in range(config.instances())]
     diff_program = os.environ.get("DIFF_VIEWER", "diff")
     with NamedTemporaryFile() as local:
       self.dump_tasks(local_tasks, local)
@@ -691,9 +694,6 @@ class UpdateCommand(Verb):
         statuses=ACTIVE_STATES, env=config.environment()))
     context.check_and_log_response(resp, err_msg="Server could not find running job to update")
     remote_tasks = [t.assignedTask.task for t in resp.result.scheduleStatusResult.tasks]
-    resp = api.populate_job_config(config)
-    context.check_and_log_response(resp, err_code=EXIT_COMMAND_FAILURE,
-        err_msg="Server could not populate job config for comparison:")
     # for determining if an update is dangerous, we estimate the scope of the change
     # by comparing number of instances to be updated, with the number of
     # instances running in the cluster.
@@ -702,7 +702,7 @@ class UpdateCommand(Verb):
     # So updating 20 instances out of 500 isn't a large change: even though 20 < 500/4;
     # but updating 20 instances when there are only 4 running is a large change.
     if context.options.instance_spec.instance == ALL_INSTANCES:
-      local_task_count = len(resp.result.populateJobResult.populatedDEPRECATED)
+      local_task_count = config.instances()
       remote_task_count = len(remote_tasks)
     else:
       local_task_count = len(context.options.instance_spec.instance)

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/66a44b07/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 cbaa67a..7c9505b 100644
--- a/src/main/python/apache/aurora/client/commands/core.py
+++ b/src/main/python/apache/aurora/client/commands/core.py
@@ -23,6 +23,7 @@ import pprint
 import subprocess
 import sys
 import time
+from copy import deepcopy
 from datetime import datetime
 from tempfile import NamedTemporaryFile
 
@@ -38,6 +39,7 @@ from apache.aurora.client.base import (
     combine_messages,
     deprecation_warning,
     die,
+    get_populated_task_config,
     handle_open,
     requires,
     synthesize_url,
@@ -285,11 +287,11 @@ def diff(job_spec, config_file):
   resp = api.populate_job_config(config)
   if resp.responseCode != ResponseCode.OK:
     die('Request failed, server responded with "%s"' % combine_messages(resp))
-  local_tasks = resp.result.populateJobResult.populatedDEPRECATED
+  local_tasks = [deepcopy(get_populated_task_config(resp)) for _ in range(config.instances())]
 
   pp = pprint.PrettyPrinter(indent=2)
   def pretty_print_task(task):
-  # The raw configuration is not interesting - we only care about what gets parsed.
+    # The raw configuration is not interesting - we only care about what gets parsed.
     task.configuration = None
     task.executorConfig = ExecutorConfig(
         name=AURORA_EXECUTOR_NAME,
@@ -703,7 +705,7 @@ def status(args, options):
 
 
 def really_update(job_spec, config_file, options):
-  def warn_if_dangerous_change(api, job_spec, config):
+  def warn_if_dangerous_change(api, config):
     # Get the current job status, so that we can check if there's anything
     # dangerous about this update.
     resp = api.query_no_configs(api.build_query(config.role(), config.name(),
@@ -711,10 +713,7 @@ def really_update(job_spec, config_file, options):
     if resp.responseCode != ResponseCode.OK:
       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' % combine_messages(resp))
-    local_task_count = len(resp.result.populateJobResult.populatedDEPRECATED)
+    local_task_count = config.instances()
     remote_task_count = len(remote_tasks)
     if (local_task_count >= 4 * remote_task_count or local_task_count <= 4 * remote_task_count
         or local_task_count == 0):

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/66a44b07/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 dad69e2..dd3f228 100644
--- a/src/test/python/apache/aurora/client/api/test_updater.py
+++ b/src/test/python/apache/aurora/client/api/test_updater.py
@@ -197,8 +197,12 @@ class UpdaterTest(TestCase):
 
   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))
+    config = deepcopy(job_config.taskConfig)
+    result = set([config])
+    resp.result = Result(populateJobResult=PopulateJobResult(
+        populatedDEPRECATED=result,
+        taskConfig=config))
+
     self._scheduler.populateJobConfig(job_config).AndReturn(resp)
 
   def expect_get_tasks(self, tasks, ignore_ids=None, response_code=ResponseCode.OK):

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/66a44b07/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 d8da68f..054e6fb 100644
--- a/src/test/python/apache/aurora/client/cli/test_diff.py
+++ b/src/test/python/apache/aurora/client/cli/test_diff.py
@@ -65,7 +65,8 @@ class TestDiffCommand(AuroraClientCommandTest):
     api.populateJobConfig.return_value = populate
     tasks = set(task.assignedTask.task for task in cls.create_scheduled_tasks())
     populate.result = Result(populateJobResult=PopulateJobResult(
-        populatedDEPRECATED=tasks
+        populatedDEPRECATED=tasks,
+        taskConfig=list(tasks)[0]
     ))
     return populate
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/66a44b07/src/test/python/apache/aurora/client/cli/test_restart.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/test_restart.py b/src/test/python/apache/aurora/client/cli/test_restart.py
index c19b21d..1653ae2 100644
--- a/src/test/python/apache/aurora/client/cli/test_restart.py
+++ b/src/test/python/apache/aurora/client/cli/test_restart.py
@@ -78,7 +78,8 @@ class TestRestartCommand(AuroraClientCommandTest):
   def setup_populate_job_config(cls, api):
     populate = cls.create_simple_success_response()
     populate.result = Result(populateJobResult=PopulateJobResult(
-        populatedDEPRECATED={TaskConfig()}
+        populatedDEPRECATED={TaskConfig()},
+        taskConfig=TaskConfig()
     ))
     api.populateJobConfig.return_value = populate
     return populate

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/66a44b07/src/test/python/apache/aurora/client/cli/test_update.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/test_update.py b/src/test/python/apache/aurora/client/cli/test_update.py
index 044e609..561dd96 100644
--- a/src/test/python/apache/aurora/client/cli/test_update.py
+++ b/src/test/python/apache/aurora/client/cli/test_update.py
@@ -182,7 +182,8 @@ class TestUpdateCommand(AuroraClientCommandTest):
         diskMb=1,
         job=JobKey(role='bozo', environment='test', name='hello')) for i in range(count)]
     populate.result = Result(populateJobResult=PopulateJobResult(
-        populatedDEPRECATED=set(configs)
+        populatedDEPRECATED=set(configs),
+        taskConfig=configs[0]
     ))
     api.populateJobConfig.return_value = populate
     return populate

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/66a44b07/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 8e346ed..81cdd11 100644
--- a/src/test/python/apache/aurora/client/commands/test_diff.py
+++ b/src/test/python/apache/aurora/client/commands/test_diff.py
@@ -104,7 +104,8 @@ class TestDiffCommand(AuroraClientCommandTest):
     api.populateJobConfig.return_value = populate
     tasks = set(task.assignedTask.task for task in cls.create_mock_scheduled_tasks())
     populate.result = Result(populateJobResult=PopulateJobResult(
-      populatedDEPRECATED=tasks
+      populatedDEPRECATED=tasks,
+      taskConfig=list(tasks)[0]
     ))
     return populate
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/66a44b07/src/test/python/apache/aurora/client/commands/test_restart.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/commands/test_restart.py b/src/test/python/apache/aurora/client/commands/test_restart.py
index 3789f3b..c71f818 100644
--- a/src/test/python/apache/aurora/client/commands/test_restart.py
+++ b/src/test/python/apache/aurora/client/commands/test_restart.py
@@ -94,6 +94,7 @@ class TestRestartCommand(AuroraClientCommandTest):
       task_config = create_autospec(spec=TaskConfig, instance=True)
       configs.append(task_config)
     populate.result.populateJobResult.populatedDEPRECATED = set(configs)
+    populate.result.populateJobResult.taskConfig = configs[0]
     return populate
 
   @classmethod

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/66a44b07/src/test/python/apache/aurora/client/commands/test_update.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/commands/test_update.py b/src/test/python/apache/aurora/client/commands/test_update.py
index 3f5f83e..0439f60 100644
--- a/src/test/python/apache/aurora/client/commands/test_update.py
+++ b/src/test/python/apache/aurora/client/commands/test_update.py
@@ -156,7 +156,8 @@ class TestUpdateCommand(AuroraClientCommandTest):
           job=JobKey(role='mchucarroll', environment='test', name='hello'))
       configs.append(task_config)
     populate.result = Result(populateJobResult=PopulateJobResult(
-        populatedDEPRECATED=set(configs)
+        populatedDEPRECATED=set(configs),
+        taskConfig=configs[0]
     ))
     return populate
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/66a44b07/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 fba08ae..bc4424b 100644
--- a/src/test/python/apache/aurora/client/test_base.py
+++ b/src/test/python/apache/aurora/client/test_base.py
@@ -15,7 +15,14 @@ import unittest
 
 from apache.aurora.client import base
 
-from gen.apache.aurora.api.ttypes import Response, ResponseCode, ResponseDetail
+from gen.apache.aurora.api.ttypes import (
+    PopulateJobResult,
+    Response,
+    ResponseCode,
+    ResponseDetail,
+    Result,
+    TaskConfig
+)
 
 
 class TestBase(unittest.TestCase):
@@ -41,3 +48,15 @@ class TestBase(unittest.TestCase):
         responseCode=ResponseCode.ERROR,
         details=[ResponseDetail(message='Error1'), ResponseDetail(message='Error2')])
     assert base.combine_messages(resp) == 'Error1, Error2'
+
+  def test_get_populated_task_config_set(self):
+    config = TaskConfig()
+    resp = Response(responseCode=ResponseCode.OK, result=Result(populateJobResult=PopulateJobResult(
+        taskConfig=config)))
+    assert config == base.get_populated_task_config(resp)
+
+  def test_get_populated_task_config_deprecated_set(self):
+    config = TaskConfig()
+    resp = Response(responseCode=ResponseCode.OK, result=Result(populateJobResult=PopulateJobResult(
+        populatedDEPRECATED=set([config]))))
+    assert config == base.get_populated_task_config(resp)