You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by se...@apache.org on 2017/10/23 21:10:29 UTC

aurora git commit: Move job environment validation to the scheduler

Repository: aurora
Updated Branches:
  refs/heads/master 5b91150fd -> a673c5b3f


Move job environment validation to the scheduler

Removed the Job environment validation from the command line client. Validation was moved to the
the scheduler side through the `allowed_job_environments` option. By default allowing any of
`devel`, `test`, `production`, and any value matching the regular expression `staging[0-9]*`.

This allows to have a consistent behavior when using the CLI and the API.

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


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

Branch: refs/heads/master
Commit: a673c5b3fbf17bbf4c0fc1e3851f7feb67ba24d5
Parents: 5b91150
Author: Mauricio Garavaglia <ma...@gmail.com>
Authored: Mon Oct 23 23:09:47 2017 +0200
Committer: Stephan Erb <se...@apache.org>
Committed: Mon Oct 23 23:09:47 2017 +0200

----------------------------------------------------------------------
 RELEASE-NOTES.md                                |  3 ++
 docs/features/multitenancy.md                   |  5 +--
 docs/reference/configuration.md                 |  2 +-
 docs/reference/scheduler-configuration.md       |  2 ++
 .../apache/aurora/scheduler/app/AppModule.java  |  8 ++++-
 .../aurora/scheduler/base/TaskTestUtil.java     |  5 +--
 .../configuration/ConfigurationManager.java     | 13 +++++++-
 src/main/python/apache/aurora/client/config.py  | 18 -----------
 .../scheduler/config/CommandLineTest.java       |  2 ++
 .../configuration/ConfigurationManagerTest.java | 34 +++++++++++++++++---
 .../aurora/scheduler/cron/quartz/CronIT.java    |  2 +-
 .../scheduler/cron/quartz/QuartzTestUtil.java   |  2 +-
 .../aurora/scheduler/thrift/ThriftIT.java       |  4 ++-
 .../python/apache/aurora/client/test_config.py  | 19 -----------
 14 files changed, 68 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/a673c5b3/RELEASE-NOTES.md
----------------------------------------------------------------------
diff --git a/RELEASE-NOTES.md b/RELEASE-NOTES.md
index 1ec6d74..76bf868 100644
--- a/RELEASE-NOTES.md
+++ b/RELEASE-NOTES.md
@@ -28,6 +28,9 @@
 - Removed the `rewriteConfigs` thrift API call in the scheduler. This was a last-ditch mechanism
   to modify scheduler state on the fly. It was considered extremely risky to use since its
   inception, and is safer to abandon due to its lack of use and likelihood for code rot.
+- Removed the Job environment validation from the command line client. Validation was moved to the
+  the scheduler side through the `allowed_job_environments` option. By default allowing any of
+  `devel`, `test`, `production`, and any value matching the regular expression `staging[0-9]*`.
 
 0.18.0
 ======

http://git-wip-us.apache.org/repos/asf/aurora/blob/a673c5b3/docs/features/multitenancy.md
----------------------------------------------------------------------
diff --git a/docs/features/multitenancy.md b/docs/features/multitenancy.md
index 301170d..32baee3 100644
--- a/docs/features/multitenancy.md
+++ b/docs/features/multitenancy.md
@@ -24,8 +24,9 @@ assignment of [quota](#preemption). If you don't know what accounts are availabl
 sysadmin.
 
 The environment component in the job key, serves as a namespace. The values for
-environment are validated in the client and the scheduler so as to allow any of `devel`, `test`,
-`production`, and any value matching the regular expression `staging[0-9]*`.
+environment are validated in the scheduler. By default allowing any of `devel`, `test`,
+`production`, and any value matching the regular expression `staging[0-9]*`. This validation can be
+changed to allow any arbitrary regular expression by setting the scheduler option `allowed_job_environments`.
 
 None of the values imply any difference in the scheduling behavior. Conventionally, the
 "environment" is set so as to indicate a certain level of stability in the behavior of the job

http://git-wip-us.apache.org/repos/asf/aurora/blob/a673c5b3/docs/reference/configuration.md
----------------------------------------------------------------------
diff --git a/docs/reference/configuration.md b/docs/reference/configuration.md
index 0231c92..f647bcf 100644
--- a/docs/reference/configuration.md
+++ b/docs/reference/configuration.md
@@ -339,7 +339,7 @@ Job Schema
   ```name``` | String | Job name. (Default: inherited from the task attribute's name)
   ```role``` | String | Job role account. Required.
   ```cluster``` | String | Cluster in which this job is scheduled. Required.
-  ```environment``` | String | Job environment, default ```devel```. Must be one of ```prod```, ```devel```, ```test``` or ```staging<number>```.
+  ```environment``` | String | Job environment, default ```devel```. By default must be one of ```prod```, ```devel```, ```test``` or ```staging<number>``` but it can be changed by the Cluster operator using the scheduler option `allowed_job_environments`.
   ```contact``` | String | Best email address to reach the owner of the job. For production jobs, this is usually a team mailing list.
   ```instances```| Integer | Number of instances (sometimes referred to as replicas or shards) of the task to create. (Default: 1)
   ```cron_schedule``` | String | Cron schedule in cron format. May only be used with non-service jobs. See [Cron Jobs](../features/cron-jobs.md) for more information. Default: None (not a cron job.)

http://git-wip-us.apache.org/repos/asf/aurora/blob/a673c5b3/docs/reference/scheduler-configuration.md
----------------------------------------------------------------------
diff --git a/docs/reference/scheduler-configuration.md b/docs/reference/scheduler-configuration.md
index 4e3f907..44ba389 100644
--- a/docs/reference/scheduler-configuration.md
+++ b/docs/reference/scheduler-configuration.md
@@ -50,6 +50,8 @@ Optional flags:
 	Allow jobs to request Mesos GPU resource.
 -allowed_container_types (default [MESOS])
 	Container types that are allowed to be used by jobs.
+-allowed_job_environments (default ^(prod|devel|test|staging\d*)$)
+	Regular expression describing the environments that are allowed to be used by jobs.
 -async_slot_stat_update_interval (default (1, mins))
 	Interval on which to try to update open slot stats.
 -async_task_stat_update_interval (default (1, hrs))

http://git-wip-us.apache.org/repos/asf/aurora/blob/a673c5b3/src/main/java/org/apache/aurora/scheduler/app/AppModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/app/AppModule.java b/src/main/java/org/apache/aurora/scheduler/app/AppModule.java
index 54d7d4c..40579db 100644
--- a/src/main/java/org/apache/aurora/scheduler/app/AppModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/app/AppModule.java
@@ -38,6 +38,7 @@ import org.apache.aurora.scheduler.app.SchedulerMain.Options.DriverKind;
 import org.apache.aurora.scheduler.async.AsyncModule;
 import org.apache.aurora.scheduler.config.CliOptions;
 import org.apache.aurora.scheduler.config.validators.PositiveNumber;
+import org.apache.aurora.scheduler.configuration.ConfigurationManager;
 import org.apache.aurora.scheduler.configuration.ConfigurationManager.ConfigurationManagerSettings;
 import org.apache.aurora.scheduler.events.PubsubEventModule;
 import org.apache.aurora.scheduler.filter.SchedulingFilterImpl;
@@ -111,6 +112,10 @@ public class AppModule extends AbstractModule {
             + "escalation threat.",
         arity = 1)
     public boolean allowContainerVolumes = false;
+
+    @Parameter(names = "-allowed_job_environments", description = "Regular expression describing "
+            + "the environments that are allowed to be used by jobs.")
+    public String allowedJobEnvironments = ConfigurationManager.DEFAULT_ALLOWED_JOB_ENVIRONMENTS;
   }
 
   private final ConfigurationManagerSettings configurationManagerSettings;
@@ -135,7 +140,8 @@ public class AppModule extends AbstractModule {
             opts.app.requireDockerUseExecutor,
             opts.main.allowGpuResource,
             opts.app.enableMesosFetcher,
-            opts.app.allowContainerVolumes),
+            opts.app.allowContainerVolumes,
+            opts.app.allowedJobEnvironments),
         opts.main.driverImpl,
         opts);
   }

http://git-wip-us.apache.org/repos/asf/aurora/blob/a673c5b3/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 60bbe39..7c223ea 100644
--- a/src/main/java/org/apache/aurora/scheduler/base/TaskTestUtil.java
+++ b/src/main/java/org/apache/aurora/scheduler/base/TaskTestUtil.java
@@ -62,7 +62,7 @@ import org.apache.mesos.v1.Protos.ExecutorInfo;
  */
 public final class TaskTestUtil {
 
-  public static final IJobKey JOB = JobKeys.from("role", "dev", "job");
+  public static final IJobKey JOB = JobKeys.from("role", "devel", "job");
   public static final TierInfo REVOCABLE_TIER =
       new TierInfo(true /* preemptible */, true /* revocable */);
   public static final TierInfo DEV_TIER =
@@ -86,7 +86,8 @@ public final class TaskTestUtil {
           true,
           true,
           true,
-          true);
+          true,
+          ConfigurationManager.DEFAULT_ALLOWED_JOB_ENVIRONMENTS);
   public static final ExecutorID EXECUTOR_ID = ExecutorID.newBuilder()
       .setValue("PLACEHOLDER")
       .build();

http://git-wip-us.apache.org/repos/asf/aurora/blob/a673c5b3/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 b7f5e35..fa2f39c 100644
--- a/src/main/java/org/apache/aurora/scheduler/configuration/ConfigurationManager.java
+++ b/src/main/java/org/apache/aurora/scheduler/configuration/ConfigurationManager.java
@@ -14,6 +14,7 @@
 package org.apache.aurora.scheduler.configuration;
 
 import java.util.List;
+import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 
 import javax.annotation.Nullable;
@@ -66,6 +67,7 @@ import static org.apache.aurora.scheduler.resources.ResourceType.RAM_MB;
 public class ConfigurationManager {
 
   public static final String DEDICATED_ATTRIBUTE = "dedicated";
+  public static final String DEFAULT_ALLOWED_JOB_ENVIRONMENTS = "^(prod|devel|test|staging\\d*)$";
 
   private interface Validator<T> {
     void validate(T value) throws TaskDescriptionException;
@@ -96,6 +98,7 @@ public class ConfigurationManager {
     private final boolean allowGpuResource;
     private final boolean enableMesosFetcher;
     private final boolean allowContainerVolumes;
+    private final Pattern allowedJobEnvironments;
 
     public ConfigurationManagerSettings(
         ImmutableSet<Container._Fields> allowedContainerTypes,
@@ -104,7 +107,8 @@ public class ConfigurationManager {
         boolean requireDockerUseExecutor,
         boolean allowGpuResource,
         boolean enableMesosFetcher,
-        boolean allowContainerVolumes) {
+        boolean allowContainerVolumes,
+        String allowedJobEnvironment) {
 
       this.allowedContainerTypes = requireNonNull(allowedContainerTypes);
       this.allowDockerParameters = allowDockerParameters;
@@ -113,6 +117,7 @@ public class ConfigurationManager {
       this.allowGpuResource = allowGpuResource;
       this.enableMesosFetcher = enableMesosFetcher;
       this.allowContainerVolumes = allowContainerVolumes;
+      this.allowedJobEnvironments = Pattern.compile(requireNonNull(allowedJobEnvironment));
     }
   }
 
@@ -179,6 +184,12 @@ public class ConfigurationManager {
       throw new TaskDescriptionException("Job key " + job.getKey() + " is invalid.");
     }
 
+    if (!settings.allowedJobEnvironments.matcher(job.getKey().getEnvironment()).matches()) {
+      throw new TaskDescriptionException(String.format(
+              "Job environment %s doesn't match: %s", job.getKey().getEnvironment(),
+              settings.allowedJobEnvironments.toString()));
+    }
+
     if (job.isSetOwner() && !UserProvidedStrings.isGoodIdentifier(job.getOwner().getUser())) {
       throw new TaskDescriptionException(
           "Job user contains illegal characters: " + job.getOwner().getUser());

http://git-wip-us.apache.org/repos/asf/aurora/blob/a673c5b3/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 70c2c98..adc8db0 100644
--- a/src/main/python/apache/aurora/client/config.py
+++ b/src/main/python/apache/aurora/client/config.py
@@ -19,7 +19,6 @@ from __future__ import print_function
 
 import functools
 import math
-import re
 import sys
 
 from pystachio import Empty
@@ -55,22 +54,6 @@ def _validate_announce_configuration(config):
       raise ValueError('Job must be dedicated in order to specify static ports!')
 
 
-STAGING_RE = re.compile(r'^staging\d*$')
-
-
-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())
-  __validate_env(env_name, 'Environment')
-
-
 UPDATE_CONFIG_MAX_FAILURES_ERROR = '''
 max_total_failures in update_config must be lesser than the job size.
 Based on your job size (%s) you should use max_total_failures <= %s.
@@ -146,7 +129,6 @@ def _validate_deprecated_config(config):
 def validate_config(config, env=None):
   _validate_update_config(config)
   _validate_announce_configuration(config)
-  _validate_environment_name(config)
   _validate_deprecated_config(config)
 
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/a673c5b3/src/test/java/org/apache/aurora/scheduler/config/CommandLineTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/config/CommandLineTest.java b/src/test/java/org/apache/aurora/scheduler/config/CommandLineTest.java
index 5b50244..f7c945d 100644
--- a/src/test/java/org/apache/aurora/scheduler/config/CommandLineTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/config/CommandLineTest.java
@@ -142,6 +142,7 @@ public class CommandLineTest {
     expected.app.requireDockerUseExecutor = false;
     expected.app.enableMesosFetcher = true;
     expected.app.allowContainerVolumes = true;
+    expected.app.allowedJobEnvironments = "^(foo|bar|zaa)$";
     expected.main.clusterName = "testing";
     expected.main.serversetPath = "testing";
     expected.main.serversetEndpointName = "testing";
@@ -294,6 +295,7 @@ public class CommandLineTest {
         "-require_docker_use_executor=false",
         "-enable_mesos_fetcher=true",
         "-allow_container_volumes=true",
+        "-allowed_job_environments=^(foo|bar|zaa)$",
         "-cluster_name=testing",
         "-serverset_path=testing",
         "-serverset_endpoint_name=testing",

http://git-wip-us.apache.org/repos/asf/aurora/blob/a673c5b3/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 265e87e..749ffea 100644
--- a/src/test/java/org/apache/aurora/scheduler/configuration/ConfigurationManagerTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/configuration/ConfigurationManagerTest.java
@@ -44,6 +44,7 @@ import org.apache.aurora.scheduler.configuration.ConfigurationManager.Configurat
 import org.apache.aurora.scheduler.configuration.ConfigurationManager.TaskDescriptionException;
 import org.apache.aurora.scheduler.mesos.TestExecutorSettings;
 import org.apache.aurora.scheduler.storage.entities.IDockerParameter;
+import org.apache.aurora.scheduler.storage.entities.IJobConfiguration;
 import org.apache.aurora.scheduler.storage.entities.ITaskConfig;
 import org.junit.Rule;
 import org.junit.Test;
@@ -124,7 +125,8 @@ public class ConfigurationManagerTest {
           true,
           false,
           true,
-          false),
+          false,
+          ConfigurationManager.DEFAULT_ALLOWED_JOB_ENVIRONMENTS),
       TaskTestUtil.TIER_MANAGER,
       TaskTestUtil.THRIFT_BACKFILL,
       TestExecutorSettings.THERMOS_EXECUTOR);
@@ -136,7 +138,8 @@ public class ConfigurationManagerTest {
           false,
           true,
           true,
-          true),
+          true,
+          ConfigurationManager.DEFAULT_ALLOWED_JOB_ENVIRONMENTS),
       TaskTestUtil.TIER_MANAGER,
       TaskTestUtil.THRIFT_BACKFILL,
       TestExecutorSettings.THERMOS_EXECUTOR);
@@ -289,7 +292,8 @@ public class ConfigurationManagerTest {
             false,
             false,
             false,
-            false),
+            false,
+            ConfigurationManager.DEFAULT_ALLOWED_JOB_ENVIRONMENTS),
         TaskTestUtil.TIER_MANAGER,
         TaskTestUtil.THRIFT_BACKFILL,
         TestExecutorSettings.THERMOS_EXECUTOR).validateAndPopulate(ITaskConfig.build(builder));
@@ -312,7 +316,8 @@ public class ConfigurationManagerTest {
                     false,
                     false,
                     false,
-                    false),
+                    false,
+                    ".+"),
             TaskTestUtil.TIER_MANAGER,
             TaskTestUtil.THRIFT_BACKFILL,
             TestExecutorSettings.THERMOS_EXECUTOR).validateAndPopulate(ITaskConfig.build(builder));
@@ -342,6 +347,27 @@ public class ConfigurationManagerTest {
     assertEquals(ImmutableSet.of("health", "http"), populated.getTaskLinks().keySet());
   }
 
+  @Test
+  public void testJobEnvironmentValidation() throws Exception {
+    JobConfiguration jobConfiguration = UNSANITIZED_JOB_CONFIGURATION.deepCopy();
+    jobConfiguration.getKey().setEnvironment("foo");
+    expectTaskDescriptionException("Job environment foo doesn't match: b.r");
+    new ConfigurationManager(
+      new ConfigurationManagerSettings(
+          ALL_CONTAINER_TYPES,
+          true,
+          ImmutableList.of(new DockerParameter("foo", "bar")),
+          false,
+          true,
+          true,
+          true,
+          "b.r"),
+      TaskTestUtil.TIER_MANAGER,
+      TaskTestUtil.THRIFT_BACKFILL,
+      TestExecutorSettings.THERMOS_EXECUTOR)
+            .validateAndPopulate(IJobConfiguration.build(jobConfiguration));
+  }
+
   private void expectTaskDescriptionException(String message) {
     expectedException.expect(TaskDescriptionException.class);
     expectedException.expectMessage(message);

http://git-wip-us.apache.org/repos/asf/aurora/blob/a673c5b3/src/test/java/org/apache/aurora/scheduler/cron/quartz/CronIT.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/cron/quartz/CronIT.java b/src/test/java/org/apache/aurora/scheduler/cron/quartz/CronIT.java
index 459d6be..1ed6a7b 100644
--- a/src/test/java/org/apache/aurora/scheduler/cron/quartz/CronIT.java
+++ b/src/test/java/org/apache/aurora/scheduler/cron/quartz/CronIT.java
@@ -59,7 +59,7 @@ import static org.junit.Assert.assertTrue;
 public class CronIT extends EasyMockTest {
   public static final CrontabEntry CRONTAB_ENTRY = CrontabEntry.parse("* * * * *");
 
-  private static final IJobKey JOB_KEY = JobKeys.from("roll", "b", "c");
+  private static final IJobKey JOB_KEY = JobKeys.from("roll", "prod", "c");
   private static final Identity IDENTITY = new Identity().setUser("user");
 
   private static final IJobConfiguration CRON_JOB = IJobConfiguration.build(

http://git-wip-us.apache.org/repos/asf/aurora/blob/a673c5b3/src/test/java/org/apache/aurora/scheduler/cron/quartz/QuartzTestUtil.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/cron/quartz/QuartzTestUtil.java b/src/test/java/org/apache/aurora/scheduler/cron/quartz/QuartzTestUtil.java
index 3c5ecd6..b7dcf3a 100644
--- a/src/test/java/org/apache/aurora/scheduler/cron/quartz/QuartzTestUtil.java
+++ b/src/test/java/org/apache/aurora/scheduler/cron/quartz/QuartzTestUtil.java
@@ -33,7 +33,7 @@ import org.quartz.JobKey;
  * Fixtures used across quartz tests.
  */
 final class QuartzTestUtil {
-  static final IJobKey AURORA_JOB_KEY = JobKeys.from("role", "env", "job");
+  static final IJobKey AURORA_JOB_KEY = JobKeys.from("role", "prod", "job");
   static final IJobConfiguration JOB = IJobConfiguration.build(
       new JobConfiguration()
           .setCronSchedule("* * * * SUN")

http://git-wip-us.apache.org/repos/asf/aurora/blob/a673c5b3/src/test/java/org/apache/aurora/scheduler/thrift/ThriftIT.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/thrift/ThriftIT.java b/src/test/java/org/apache/aurora/scheduler/thrift/ThriftIT.java
index 2cd19d5..1f30bfa 100644
--- a/src/test/java/org/apache/aurora/scheduler/thrift/ThriftIT.java
+++ b/src/test/java/org/apache/aurora/scheduler/thrift/ThriftIT.java
@@ -48,6 +48,7 @@ import org.apache.aurora.scheduler.app.ServiceGroupMonitor;
 import org.apache.aurora.scheduler.app.local.FakeNonVolatileStorage;
 import org.apache.aurora.scheduler.base.TaskTestUtil;
 import org.apache.aurora.scheduler.config.CliOptions;
+import org.apache.aurora.scheduler.configuration.ConfigurationManager;
 import org.apache.aurora.scheduler.configuration.ConfigurationManager.ConfigurationManagerSettings;
 import org.apache.aurora.scheduler.configuration.executor.ExecutorSettings;
 import org.apache.aurora.scheduler.cron.quartz.CronModule;
@@ -177,7 +178,8 @@ public class ThriftIT extends EasyMockTest {
         false,
         true,
         true,
-        false);
+        false,
+        ConfigurationManager.DEFAULT_ALLOWED_JOB_ENVIRONMENTS);
 
     createThrift(configurationManagerSettings);
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/a673c5b3/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 042372e..3d5289a 100644
--- a/src/test/python/apache/aurora/client/test_config.py
+++ b/src/test/python/apache/aurora/client/test_config.py
@@ -141,25 +141,6 @@ 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():
-  base_job = Job(
-      name='hello_world', role='john_doe', cluster='test-cluster',
-      task=Task(name='main', processes=[],
-                resources=Resources(cpu=0.1, ram=64 * MB, disk=64 * MB)))
-
-  with pytest.raises(ValueError):
-    config._validate_environment_name(AuroraConfig(base_job))
-  for env_name in GOOD_ENV:
-    config._validate_environment_name(AuroraConfig(base_job(environment=env_name)))
-  for env_name in BAD_ENV:
-    with pytest.raises(ValueError):
-      config._validate_environment_name(AuroraConfig(base_job(environment=env_name)))
-
-
 def test_dedicated_portmap():
   base_job = Job(
       name='hello_world', role='john_doe', cluster='test-cluster',