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/08/26 00:57:43 UTC

aurora git commit: Revocable: schema changes.

Repository: aurora
Updated Branches:
  refs/heads/master f58cbf934 -> 506d0cdbc


Revocable: schema changes.

Bugs closed: AURORA-1414

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


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

Branch: refs/heads/master
Commit: 506d0cdbcb558fe43cab60ced8b613cfddc54b7f
Parents: f58cbf9
Author: Maxim Khutornenko <ma...@apache.org>
Authored: Tue Aug 25 15:55:20 2015 -0700
Committer: Maxim Khutornenko <ma...@apache.org>
Committed: Tue Aug 25 15:55:20 2015 -0700

----------------------------------------------------------------------
 .../thrift/org/apache/aurora/gen/api.thrift     |  2 ++
 .../apache/aurora/scheduler/TierManager.java    |  4 +---
 .../aurora/scheduler/base/TaskTestUtil.java     |  1 +
 .../configuration/ConfigurationManager.java     |  4 ++++
 .../storage/db/views/DbTaskConfig.java          |  2 ++
 src/main/python/apache/aurora/client/config.py  | 25 ++++++++++++++++----
 .../python/apache/aurora/config/schema/base.py  |  1 +
 src/main/python/apache/aurora/config/thrift.py  |  1 +
 .../scheduler/storage/db/TaskConfigMapper.xml   |  3 +++
 .../aurora/scheduler/storage/db/schema.sql      |  3 ++-
 .../aurora/scheduler/TierManagerTest.java       |  2 +-
 .../configuration/ConfigurationManagerTest.java | 15 ++++++++++--
 .../apache/aurora/client/cli/test_create.py     |  5 +---
 .../python/apache/aurora/client/test_config.py  | 25 ++++++++++++++++----
 .../python/apache/aurora/config/test_thrift.py  |  7 ++++++
 15 files changed, 80 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/506d0cdb/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 f792be0..21137bb 100644
--- a/api/src/main/thrift/org/apache/aurora/gen/api.thrift
+++ b/api/src/main/thrift/org/apache/aurora/gen/api.thrift
@@ -251,6 +251,8 @@ struct TaskConfig {
  13: i32 maxTaskFailures
  /** Whether this is a production task, which can preempt. */
  18: optional bool production
+ /** Task tier type. */
+ 30: optional string tier
 
  20: set<Constraint> constraints
  /** a list of named ports this task requests */

http://git-wip-us.apache.org/repos/asf/aurora/blob/506d0cdb/src/main/java/org/apache/aurora/scheduler/TierManager.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/TierManager.java b/src/main/java/org/apache/aurora/scheduler/TierManager.java
index ebfad97..652afec 100644
--- a/src/main/java/org/apache/aurora/scheduler/TierManager.java
+++ b/src/main/java/org/apache/aurora/scheduler/TierManager.java
@@ -17,8 +17,7 @@ import org.apache.aurora.scheduler.storage.entities.ITaskConfig;
 
 /**
  * Translates job tier configuration into a set of task traits/attributes.
- * TODO(maxim): Implement external configuration support defined here:
- * https://docs.google.com/document/d/1gexe2uM_9gjsV62cMmX0VjH85Uokko21vEoENY2jjF0
+ * TODO(maxim): Implement external configuration support: AURORA-1443.
  */
 public interface TierManager {
 
@@ -34,7 +33,6 @@ public interface TierManager {
 
     @Override
     public TierInfo getTier(ITaskConfig taskConfig) {
-      // TODO(maxim): Implement when schema changes are defined.
       return new TierInfo(false);
     }
   }

http://git-wip-us.apache.org/repos/asf/aurora/blob/506d0cdb/src/main/java/org/apache/aurora/scheduler/base/TaskTestUtil.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/base/TaskTestUtil.java b/src/main/java/org/apache/aurora/scheduler/base/TaskTestUtil.java
index 1903315..d7f3c60 100644
--- a/src/main/java/org/apache/aurora/scheduler/base/TaskTestUtil.java
+++ b/src/main/java/org/apache/aurora/scheduler/base/TaskTestUtil.java
@@ -63,6 +63,7 @@ public final class TaskTestUtil {
         .setPriority(1)
         .setMaxTaskFailures(-1)
         .setProduction(true)
+        .setTier("tier-" + job.getEnvironment())
         .setConstraints(ImmutableSet.of(
             new Constraint(
                 "valueConstraint",

http://git-wip-us.apache.org/repos/asf/aurora/blob/506d0cdb/src/main/java/org/apache/aurora/scheduler/configuration/ConfigurationManager.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/configuration/ConfigurationManager.java b/src/main/java/org/apache/aurora/scheduler/configuration/ConfigurationManager.java
index d103d19..05e8b71 100644
--- a/src/main/java/org/apache/aurora/scheduler/configuration/ConfigurationManager.java
+++ b/src/main/java/org/apache/aurora/scheduler/configuration/ConfigurationManager.java
@@ -287,6 +287,10 @@ public final class ConfigurationManager {
           "Environment contains illegal characters: " + config.getEnvironment());
     }
 
+    if (config.isSetTier() && !isGoodIdentifier(config.getTier())) {
+      throw new TaskDescriptionException("Tier contains illegal characters: " + config.getTier());
+    }
+
     if (config.isSetJob()) {
       if (!JobKeys.isValid(config.getJob())) {
         // Job key is set but invalid

http://git-wip-us.apache.org/repos/asf/aurora/blob/506d0cdb/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbTaskConfig.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbTaskConfig.java b/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbTaskConfig.java
index 956c508..9879836 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbTaskConfig.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbTaskConfig.java
@@ -48,6 +48,7 @@ public final class DbTaskConfig {
   private ExecutorConfig executorConfig;
   private List<Metadata> metadata;
   private DbContainer container;
+  private String tier;
 
   private DbTaskConfig() {
   }
@@ -69,6 +70,7 @@ public final class DbTaskConfig {
         .setPriority(priority)
         .setMaxTaskFailures(maxTaskFailures)
         .setProduction(production)
+        .setTier(tier)
         .setConstraints(constraints.stream()
             .map(DbConstraint::toThrift)
             .collect(GuavaUtils.toImmutableSet()))

http://git-wip-us.apache.org/repos/asf/aurora/blob/506d0cdb/src/main/python/apache/aurora/client/config.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/config.py b/src/main/python/apache/aurora/client/config.py
index 59703ef..f5ac084 100644
--- a/src/main/python/apache/aurora/client/config.py
+++ b/src/main/python/apache/aurora/client/config.py
@@ -22,6 +22,8 @@ import math
 import re
 import sys
 
+from pystachio import Empty
+
 from apache.aurora.client import binding_helper
 from apache.aurora.client.base import die
 from apache.aurora.config import AuroraConfig
@@ -55,13 +57,25 @@ def _validate_announce_configuration(config):
 STAGING_RE = re.compile(r'^staging\d*$')
 
 
+#TODO(maxim): Merge env and tier and move definitions to scheduler: AURORA-1443.
+def __validate_env(name, config_name):
+  if STAGING_RE.match(name):
+    return
+  if name not in ('prod', 'devel', 'test'):
+    raise ValueError('%s should be one of "prod", "devel", "test" or '
+                     'staging<number>!  Got %s' % (config_name, name))
+
+
 def _validate_environment_name(config):
   env_name = str(config.raw().environment())
-  if STAGING_RE.match(env_name):
-    return
-  if env_name not in ('prod', 'devel', 'test'):
-    raise ValueError('Environment name should be one of "prod", "devel", "test" or '
-                     'staging<number>!  Got %s' % env_name)
+  __validate_env(env_name, 'Environment')
+
+
+def _validate_tier(config):
+  tier_raw = config.raw().tier()
+  tier_name = str(tier_raw) if tier_raw is not Empty else None
+  if tier_name is not None:
+    __validate_env(tier_name, 'Tier')
 
 
 UPDATE_CONFIG_MAX_FAILURES_ERROR = '''
@@ -114,6 +128,7 @@ def validate_config(config, env=None):
   _validate_update_config(config)
   _validate_announce_configuration(config)
   _validate_environment_name(config)
+  _validate_tier(config)
 
 
 class GlobalHookRegistry(object):

http://git-wip-us.apache.org/repos/asf/aurora/blob/506d0cdb/src/main/python/apache/aurora/config/schema/base.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/config/schema/base.py b/src/main/python/apache/aurora/config/schema/base.py
index 214d559..f8a1f05 100644
--- a/src/main/python/apache/aurora/config/schema/base.py
+++ b/src/main/python/apache/aurora/config/schema/base.py
@@ -110,6 +110,7 @@ class MesosJob(Struct):
   instances     = Default(Integer, 1)
   task          = Required(Task)
   announce      = Announcer
+  tier          = String
 
   cron_schedule = String
   cron_collision_policy = Default(String, "KILL_EXISTING")

http://git-wip-us.apache.org/repos/asf/aurora/blob/506d0cdb/src/main/python/apache/aurora/config/thrift.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/config/thrift.py b/src/main/python/apache/aurora/config/thrift.py
index adf53bb..f69977c 100644
--- a/src/main/python/apache/aurora/config/thrift.py
+++ b/src/main/python/apache/aurora/config/thrift.py
@@ -197,6 +197,7 @@ def convert(job, metadata=frozenset(), ports=frozenset()):
   task.maxTaskFailures = fully_interpolated(job.max_task_failures())
   task.priority = fully_interpolated(job.priority())
   task.contactEmail = not_empty_or(job.contact(), None)
+  task.tier = not_empty_or(job.tier(), None)
 
   # Add metadata to a task, to display in the scheduler UI.
   task.metadata = frozenset(Metadata(key=str(key), value=str(value)) for key, value in metadata)

http://git-wip-us.apache.org/repos/asf/aurora/blob/506d0cdb/src/main/resources/org/apache/aurora/scheduler/storage/db/TaskConfigMapper.xml
----------------------------------------------------------------------
diff --git a/src/main/resources/org/apache/aurora/scheduler/storage/db/TaskConfigMapper.xml b/src/main/resources/org/apache/aurora/scheduler/storage/db/TaskConfigMapper.xml
index bfad339..c70e859 100644
--- a/src/main/resources/org/apache/aurora/scheduler/storage/db/TaskConfigMapper.xml
+++ b/src/main/resources/org/apache/aurora/scheduler/storage/db/TaskConfigMapper.xml
@@ -32,6 +32,7 @@
       contact_email,
       executor_name,
       executor_data,
+      tier
     ) VALUES (
       (
         SELECT ID
@@ -51,6 +52,7 @@
       #{config.contactEmail},
       #{config.executorConfig.name},
       #{config.executorConfig.data},
+      #{config.tier}
     )
   </insert>
 
@@ -167,6 +169,7 @@
       c.contact_email AS contact_email,
       c.executor_name AS executor_name,
       c.executor_data AS executor_data,
+      c.tier AS tier,
       j.role AS j_role,
       j.environment AS j_environment,
       j.name AS j_name,

http://git-wip-us.apache.org/repos/asf/aurora/blob/506d0cdb/src/main/resources/org/apache/aurora/scheduler/storage/db/schema.sql
----------------------------------------------------------------------
diff --git a/src/main/resources/org/apache/aurora/scheduler/storage/db/schema.sql b/src/main/resources/org/apache/aurora/scheduler/storage/db/schema.sql
index 7634047..d971aa1 100644
--- a/src/main/resources/org/apache/aurora/scheduler/storage/db/schema.sql
+++ b/src/main/resources/org/apache/aurora/scheduler/storage/db/schema.sql
@@ -187,7 +187,8 @@ CREATE TABLE task_configs(
   production BOOLEAN NOT NULL,
   contact_email VARCHAR,
   executor_name VARCHAR NOT NULL,
-  executor_data VARCHAR NOT NULL
+  executor_data VARCHAR NOT NULL,
+  tier VARCHAR
 );
 
 CREATE TABLE task_constraints(

http://git-wip-us.apache.org/repos/asf/aurora/blob/506d0cdb/src/test/java/org/apache/aurora/scheduler/TierManagerTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/TierManagerTest.java b/src/test/java/org/apache/aurora/scheduler/TierManagerTest.java
index 37e19ac..eb02071 100644
--- a/src/test/java/org/apache/aurora/scheduler/TierManagerTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/TierManagerTest.java
@@ -23,7 +23,7 @@ import static org.junit.Assert.assertEquals;
 public class TierManagerTest {
 
   @Test
-  public void testIsRevocable() {
+  public void testIsNotRevocable() {
     TierInfo expected = new TierInfo(false);
     assertEquals(expected, new TierManagerImpl().getTier(ITaskConfig.build(new TaskConfig())));
   }

http://git-wip-us.apache.org/repos/asf/aurora/blob/506d0cdb/src/test/java/org/apache/aurora/scheduler/configuration/ConfigurationManagerTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/configuration/ConfigurationManagerTest.java b/src/test/java/org/apache/aurora/scheduler/configuration/ConfigurationManagerTest.java
index b231827..f3b62cc 100644
--- a/src/test/java/org/apache/aurora/scheduler/configuration/ConfigurationManagerTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/configuration/ConfigurationManagerTest.java
@@ -28,6 +28,7 @@ import org.apache.aurora.gen.LimitConstraint;
 import org.apache.aurora.gen.TaskConfig;
 import org.apache.aurora.gen.TaskConstraint;
 import org.apache.aurora.gen.ValueConstraint;
+import org.apache.aurora.scheduler.configuration.ConfigurationManager.TaskDescriptionException;
 import org.apache.aurora.scheduler.storage.entities.ITaskConfig;
 import org.junit.Test;
 
@@ -110,11 +111,21 @@ public class ConfigurationManagerTest {
     assertTrue(copy.isSetKey());
   }
 
-  @Test(expected = ConfigurationManager.TaskDescriptionException.class)
-  public void testBadContainerConfig() throws ConfigurationManager.TaskDescriptionException {
+  @Test(expected = TaskDescriptionException.class)
+  public void testBadContainerConfig() throws TaskDescriptionException {
     TaskConfig taskConfig = CONFIG_WITH_CONTAINER.deepCopy();
     taskConfig.getContainer().getDocker().setImage(null);
 
     ConfigurationManager.validateAndPopulate(ITaskConfig.build(taskConfig));
   }
+
+  @Test(expected = TaskDescriptionException.class)
+  public void testInvalidTier() throws TaskDescriptionException {
+    ITaskConfig config = ITaskConfig.build(UNSANITIZED_JOB_CONFIGURATION.deepCopy().getTaskConfig()
+        .setJobName("job")
+        .setEnvironment("env")
+        .setTier("pr/d"));
+
+    ConfigurationManager.validateAndPopulate(config);
+  }
 }

http://git-wip-us.apache.org/repos/asf/aurora/blob/506d0cdb/src/test/python/apache/aurora/client/cli/test_create.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/test_create.py b/src/test/python/apache/aurora/client/cli/test_create.py
index 69039b6..8c27e2b 100644
--- a/src/test/python/apache/aurora/client/cli/test_create.py
+++ b/src/test/python/apache/aurora/client/cli/test_create.py
@@ -404,10 +404,7 @@ class TestClientCreateCommand(AuroraClientCommandTest):
             fp.name])
         assert result == EXIT_INVALID_CONFIGURATION
       assert mock_context.get_out() == []
-      assert mock_context.get_err() == [
-            "Error loading configuration: "
-            "TypeCheck(FAILED): MesosJob[update_config] failed: "
-            "UpdateConfig[batch_size] failed: u'{{TEST_BATCH}}' not an integer"]
+      assert "Error loading configuration: TypeCheck(FAILED):" in mock_context.get_err()[0]
 
   def test_create_cron_job_fails(self):
     """Test a cron job is not accepted."""

http://git-wip-us.apache.org/repos/asf/aurora/blob/506d0cdb/src/test/python/apache/aurora/client/test_config.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/test_config.py b/src/test/python/apache/aurora/client/test_config.py
index 986061b..37459f5 100644
--- a/src/test/python/apache/aurora/client/test_config.py
+++ b/src/test/python/apache/aurora/client/test_config.py
@@ -138,9 +138,11 @@ def test_include():
           get_aurora_config('hello_world', hello_include_fname_fp)
 
 
+BAD_ENV = ('Prod', ' prod', 'prod ', 'tEst', 'production', 'staging 2', 'stagingA')
+GOOD_ENV = ('prod', 'devel', 'test', 'staging', 'staging001', 'staging1', 'staging1234')
+
+
 def test_environment_names():
-  BAD = ('Prod', ' prod', 'prod ', 'tEst', 'production', 'staging 2', 'stagingA')
-  GOOD = ('prod', 'devel', 'test', 'staging', 'staging001', 'staging1', 'staging1234')
   base_job = Job(
       name='hello_world', role='john_doe', cluster='test-cluster',
       task=Task(name='main', processes=[],
@@ -148,13 +150,28 @@ def test_environment_names():
 
   with pytest.raises(ValueError):
     config._validate_environment_name(AuroraConfig(base_job))
-  for env_name in GOOD:
+  for env_name in GOOD_ENV:
     config._validate_environment_name(AuroraConfig(base_job(environment=env_name)))
-  for env_name in BAD:
+  for env_name in BAD_ENV:
     with pytest.raises(ValueError):
       config._validate_environment_name(AuroraConfig(base_job(environment=env_name)))
 
 
+def test_tier_names():
+  base_job = Job(
+      name='hello_world', role='john_doe', cluster='test-cluster',
+      task=Task(name='main', processes=[]))
+
+  # Make sure empty value does not raise.
+  config._validate_tier(AuroraConfig(base_job))
+
+  for tier in GOOD_ENV:
+    config._validate_tier(AuroraConfig(base_job(tier=tier)))
+  for tier in BAD_ENV:
+    with pytest.raises(ValueError):
+      config._validate_tier(AuroraConfig(base_job(tier=tier)))
+
+
 def test_dedicated_portmap():
   base_job = Job(
       name='hello_world', role='john_doe', cluster='test-cluster',

http://git-wip-us.apache.org/repos/asf/aurora/blob/506d0cdb/src/test/python/apache/aurora/config/test_thrift.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/config/test_thrift.py b/src/test/python/apache/aurora/config/test_thrift.py
index 061864e..0c0432c 100644
--- a/src/test/python/apache/aurora/config/test_thrift.py
+++ b/src/test/python/apache/aurora/config/test_thrift.py
@@ -71,6 +71,13 @@ def test_simple_config():
   assert tti.constraints == set()
   assert tti.metadata == set()
   assert tti.environment == HELLO_WORLD.environment().get()
+  assert tti.tier is None
+
+
+def test_config_with_tier():
+  config = HELLO_WORLD(tier='devel')
+  job = convert_pystachio_to_thrift(config)
+  assert job.taskConfig.tier == 'devel'
 
 
 def test_docker_with_parameters():