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 2016/02/01 23:24:30 UTC

aurora git commit: Improve --read-json to handle multi-job files

Repository: aurora
Updated Branches:
  refs/heads/master 021b2dd65 -> 166ca416d


Improve --read-json to handle multi-job files

Still handles the old --read-json behavior of expecting a single job,
but adds the ability to read files with a {"jobs": [job1, job2, ...]}
schema like the pystachio format.

Also adds --read-json to the `aurora config load` command, as it is
now useful there.

Json configs are now loaded in a way that is much closer to the
pystachio one, so the config loader will no longer ignore unknown
fields.

Bugs closed: AURORA-1577

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


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

Branch: refs/heads/master
Commit: 166ca416d8b837b8c409b23a1c953dabd9b8ace8
Parents: 021b2dd
Author: Benjamin Staffin <be...@gmail.com>
Authored: Mon Feb 1 14:24:13 2016 -0800
Committer: Bill Farner <wf...@apache.org>
Committed: Mon Feb 1 14:24:13 2016 -0800

----------------------------------------------------------------------
 NEWS                                            |  7 ++++
 .../python/apache/aurora/client/cli/config.py   |  9 +++--
 .../python/apache/aurora/config/__init__.py     | 14 ++++---
 src/main/python/apache/aurora/config/loader.py  | 13 ++++++-
 .../python/apache/aurora/config/test_loader.py  | 39 +++++++++++++++++---
 5 files changed, 66 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/166ca416/NEWS
----------------------------------------------------------------------
diff --git a/NEWS b/NEWS
index 39e6a0b..ef1e75f 100644
--- a/NEWS
+++ b/NEWS
@@ -1,3 +1,10 @@
+0.13.0
+------
+- When using --read-json, aurora can now load multiple jobs from one json file,
+  similar to the usual pystachio structure: `{"jobs": [job1, job2, ...]}`. The
+  older single-job json format is also still supported.
+- `aurora config list` command now supports `--read-json`
+
 0.12.0
 ------
 - Removed the deprecated field 'ConfigGroup.instanceIds' from the API.

http://git-wip-us.apache.org/repos/asf/aurora/blob/166ca416/src/main/python/apache/aurora/client/cli/config.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/config.py b/src/main/python/apache/aurora/client/cli/config.py
index 73b5562..0a7c832 100644
--- a/src/main/python/apache/aurora/client/cli/config.py
+++ b/src/main/python/apache/aurora/client/cli/config.py
@@ -21,7 +21,7 @@ from __future__ import print_function
 
 from apache.aurora.client.cli import EXIT_COMMAND_FAILURE, EXIT_OK, Noun, Verb
 from apache.aurora.client.cli.context import AuroraCommandContext
-from apache.aurora.client.cli.options import BIND_OPTION, CONFIG_ARGUMENT
+from apache.aurora.client.cli.options import BIND_OPTION, CONFIG_ARGUMENT, JSON_READ_OPTION
 from apache.aurora.client.config import AuroraConfig
 from apache.aurora.config.loader import AuroraConfigLoader
 
@@ -36,7 +36,7 @@ class ListJobsCommand(Verb):
     return "List all of the jobs defined in a configuration file"
 
   def get_options(self):
-    return [BIND_OPTION, CONFIG_ARGUMENT]
+    return [BIND_OPTION, CONFIG_ARGUMENT, JSON_READ_OPTION]
 
   def execute(self, context):
     def maybe_bind(j):
@@ -47,7 +47,10 @@ class ListJobsCommand(Verb):
           job.name().get()])
 
     try:
-      env = AuroraConfigLoader.load(context.options.config_file)
+      if context.options.read_json:
+        env = AuroraConfigLoader.load_json(context.options.config_file)
+      else:
+        env = AuroraConfigLoader.load(context.options.config_file)
     except (AuroraConfig.Error, AuroraConfigLoader.Error, ValueError) as e:
       context.print_err("Error loading configuration file: %s" % e)
       return EXIT_COMMAND_FAILURE

http://git-wip-us.apache.org/repos/asf/aurora/blob/166ca416/src/main/python/apache/aurora/config/__init__.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/config/__init__.py b/src/main/python/apache/aurora/config/__init__.py
index 665e2cd..65923be 100644
--- a/src/main/python/apache/aurora/config/__init__.py
+++ b/src/main/python/apache/aurora/config/__init__.py
@@ -122,14 +122,18 @@ class AuroraConfig(object):
         cls, filename, name=None, bindings=None,
         select_cluster=None, select_role=None, select_env=None):
     # TODO(atollenaere): should take a JobKey when non-jobkey interface is deprecated
-    job = AuroraConfigLoader.load_json(filename)
-    return cls.apply_plugins(cls(job.bind(*bindings) if bindings else job))
+    env = AuroraConfigLoader.load_json(filename)
+    return cls.apply_plugins(
+        cls(cls.pick(env, name, bindings, select_cluster, select_role, select_env)), env)
 
   @classmethod
-  def loads_json(cls, string, name=None, bindings=None, select_cluster=None, select_env=None):
+  def loads_json(
+        cls, string, name=None, bindings=None,
+        select_cluster=None, select_role=None, select_env=None):
     # TODO(atollenaere): should take a JobKey when non-jobkey interface is deprecated
-    job = AuroraConfigLoader.loads_json(string)
-    return cls.apply_plugins(cls(job.bind(*bindings) if bindings else job))
+    env = AuroraConfigLoader.loads_json(string)
+    return cls.apply_plugins(
+        cls(cls.pick(env, name, bindings, select_cluster, select_role, select_env)), env)
 
   @classmethod
   def validate_job(cls, job):

http://git-wip-us.apache.org/repos/asf/aurora/blob/166ca416/src/main/python/apache/aurora/config/loader.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/config/loader.py b/src/main/python/apache/aurora/config/loader.py
index a967b9d..a6ce080 100644
--- a/src/main/python/apache/aurora/config/loader.py
+++ b/src/main/python/apache/aurora/config/loader.py
@@ -12,6 +12,7 @@
 # limitations under the License.
 #
 
+import json
 import pkgutil
 
 from pystachio.config import Config as PystachioConfig
@@ -65,11 +66,19 @@ class AuroraConfigLoader(PystachioConfig):
   @classmethod
   def load_json(cls, filename):
     with open(filename) as fp:
-      return base_schema.Job.json_load(fp)
+      return cls.loads_json(fp.read())
 
   @classmethod
   def loads_json(cls, string):
-    return base_schema.Job.json_loads(string)
+    parsed = json.loads(string)
+    if 'jobs' not in parsed:
+      # Convert the legacy single-job format
+      parsed = {'jobs': [parsed]}
+
+    parsed.update({
+      'jobs': [base_schema.Job(x) for x in parsed.get('jobs')]
+    })
+    return parsed
 
 
 AuroraConfigLoader.flush_schemas()

http://git-wip-us.apache.org/repos/asf/aurora/blob/166ca416/src/test/python/apache/aurora/config/test_loader.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/config/test_loader.py b/src/test/python/apache/aurora/config/test_loader.py
index 9789255..25c99ec 100644
--- a/src/test/python/apache/aurora/config/test_loader.py
+++ b/src/test/python/apache/aurora/config/test_loader.py
@@ -27,8 +27,7 @@ BAD_MESOS_CONFIG = """
 3 2 1 3 2 4 2 3
 """
 
-
-MESOS_CONFIG = """
+JOB1 = """
 HELLO_WORLD = MesosJob(
   name = 'hello_world',
   role = 'john_doe',
@@ -39,9 +38,29 @@ HELLO_WORLD = MesosJob(
     resources = Resources(cpu = 0.1, ram = 64 * 1048576, disk = 64 * 1048576),
   )
 )
+"""
+
+JOB2 = """
+OTHERJOB = MesosJob(
+  name = 'otherjob',
+  role = 'john_doe',
+  cluster = 'smf1-test',
+  task = Task(
+    name = 'main',
+    processes = [Process(name = 'otherthing', cmdline = 'echo {{mesos.instance}}')],
+    resources = Resources(cpu = 0.1, ram = 64 * 1048576, disk = 64 * 1048576),
+  )
+)
+"""
+
+MESOS_CONFIG = JOB1 + """
 jobs = [HELLO_WORLD]
 """
 
+MESOS_CONFIG_MULTI = JOB1 + JOB2 + """
+jobs = [HELLO_WORLD, OTHERJOB]
+"""
+
 
 def test_enoent():
   nonexistent_file = tempfile.mktemp()
@@ -59,21 +78,29 @@ def test_filter_schema():
   job_dict = env['jobs'][0].get()
   job_dict['unknown_attribute'] = 'foo bar baz'
   job_json_string = json.dumps(job_dict)
-  # If this fails, will raise an InvalidConfigError or other exception and fail the test.
-  AuroraConfigLoader.loads_json(job_json_string)
+  with pytest.raises(AttributeError):
+    AuroraConfigLoader.loads_json(job_json_string)
 
 
 def test_empty_config():
   AuroraConfigLoader.load(BytesIO())
 
 
-def test_load_json():
+def test_load_json_single():
   env = AuroraConfigLoader.load(BytesIO(MESOS_CONFIG))
   job = env['jobs'][0]
-  new_job = AuroraConfigLoader.loads_json(json.dumps(job.get()))
+  new_job = AuroraConfigLoader.loads_json(json.dumps(job.get()))['jobs'][0]
   assert new_job == job
 
 
+def test_load_json_multi():
+  env = AuroraConfigLoader.load(BytesIO(MESOS_CONFIG_MULTI))
+  jobs = env['jobs']
+  json_env = AuroraConfigLoader.loads_json(json.dumps({'jobs': [job.get() for job in jobs]}))
+  json_jobs = json_env['jobs']
+  assert jobs == json_jobs
+
+
 def test_load():
   with temporary_file() as fp:
     fp.write(MESOS_CONFIG)