You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by jc...@apache.org on 2016/01/20 19:18:51 UTC

aurora git commit: Introduces -default_docker_parameters scheduler flag.

Repository: aurora
Updated Branches:
  refs/heads/master 749f83502 -> f02872529


Introduces -default_docker_parameters scheduler flag.

This flag allows cluster administrators to set arbitrary
Docker parameters which will apply to all jobs.

Also cleans up some of the existing unit tests around task config.

Bugs closed: AURORA-1575

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


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

Branch: refs/heads/master
Commit: f02872529e71dc10283ac09f845bd03e737cddec
Parents: 749f835
Author: George Sirois <ge...@gmail.com>
Authored: Wed Jan 20 12:18:32 2016 -0600
Committer: Joshua Cohen <jc...@apache.org>
Committed: Wed Jan 20 12:18:32 2016 -0600

----------------------------------------------------------------------
 NEWS                                            |  3 +
 README.md                                       |  2 +-
 .../common/args/parsers/MultimapParser.java     | 65 +++++++++++++++++
 docs/deploying-aurora-scheduler.md              | 24 ++++++-
 .../apache/aurora/scheduler/app/AppModule.java  | 10 ++-
 .../aurora/scheduler/base/TaskTestUtil.java     |  3 +-
 .../configuration/ConfigurationManager.java     | 24 +++++--
 .../configuration/ConfigurationManagerTest.java | 74 +++++++++++++++++++-
 8 files changed, 190 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/f0287252/NEWS
----------------------------------------------------------------------
diff --git a/NEWS b/NEWS
index 3937108..37a7a79 100644
--- a/NEWS
+++ b/NEWS
@@ -26,6 +26,9 @@
   made redundant: `logtostderr`, `alsologtostderr`, `vlog`, `vmodule`, and `use_glog_formatter`.
 - Added support for configuring Mesos role by passing `-mesos_role` to Aurora scheduler at start time.
   This enables resource reservation for Aurora when running in a shared Mesos cluster.
+- Added new scheduler flag `-default_docker_parameters` to allow a cluster operator to specify a
+  universal set of parameters that should be used for every container that does not have parameters
+  explicitly configured at the job level.
 
 0.11.0
 ------

http://git-wip-us.apache.org/repos/asf/aurora/blob/f0287252/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index fc8642f..da00841 100644
--- a/README.md
+++ b/README.md
@@ -61,7 +61,7 @@ Are you using Aurora too?  Let us know, or submit a patch to join the list!
 
 
 ## Getting Help
-If you have questions that aren't answered in our [doucmentation](https://aurora.apache.org/documentation/latest/), you can reach out to one of our [mailing lists](https://aurora.apache.org/community/). We're also often available in IRC: #aurora on
+If you have questions that aren't answered in our [documentation](https://aurora.apache.org/documentation/latest/), you can reach out to one of our [mailing lists](https://aurora.apache.org/community/). We're also often available in IRC: #aurora on
 [irc.freenode.net](http://webchat.freenode.net/?channels=#aurora).
 
 You can also file bugs/issues in our [JIRA queue](http://issues.apache.org/jira/browse/AURORA).

http://git-wip-us.apache.org/repos/asf/aurora/blob/f0287252/commons/src/main/java/org/apache/aurora/common/args/parsers/MultimapParser.java
----------------------------------------------------------------------
diff --git a/commons/src/main/java/org/apache/aurora/common/args/parsers/MultimapParser.java b/commons/src/main/java/org/apache/aurora/common/args/parsers/MultimapParser.java
new file mode 100644
index 0000000..0c44614
--- /dev/null
+++ b/commons/src/main/java/org/apache/aurora/common/args/parsers/MultimapParser.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.aurora.common.args.parsers;
+
+import java.lang.reflect.Type;
+import java.util.List;
+
+import com.google.common.base.Splitter;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMultimap;
+import com.google.common.collect.Multimap;
+import com.google.common.reflect.TypeToken;
+
+import org.apache.aurora.common.args.ArgParser;
+import org.apache.aurora.common.args.Parser;
+import org.apache.aurora.common.args.ParserOracle;
+import org.apache.aurora.common.args.Parsers;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+/**
+ * Multimap parser.
+ */
+@ArgParser
+public class MultimapParser extends TypeParameterizedParser<Multimap<?, ?>> {
+
+  private static final Splitter KEY_VALUE_SPLITTER =
+      Splitter.on("=").trimResults().omitEmptyStrings();
+
+  public MultimapParser() {
+    super(2);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  Multimap<?, ?> doParse(ParserOracle parserOracle, String raw, List<Type> typeParams) {
+    Type keyType = typeParams.get(0);
+    Parser<?> keyParser = parserOracle.get(TypeToken.of(keyType));
+
+    Type valueType = typeParams.get(1);
+    Parser<?> valueParser = parserOracle.get(TypeToken.of(valueType));
+
+    ImmutableMultimap.Builder<Object, Object> multimapBuilder = ImmutableMultimap.builder();
+    for (String keyAndValue : Parsers.MULTI_VALUE_SPLITTER.split(raw)) {
+      List<String> fields = ImmutableList.copyOf(KEY_VALUE_SPLITTER.split(keyAndValue));
+      checkArgument(fields.size() == 2, "Failed to parse key/value pair: " + keyAndValue);
+      multimapBuilder.put(
+          keyParser.parse(parserOracle, keyType, fields.get(0)),
+          valueParser.parse(parserOracle, valueType, fields.get(1)));
+    }
+
+    return multimapBuilder.build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/f0287252/docs/deploying-aurora-scheduler.md
----------------------------------------------------------------------
diff --git a/docs/deploying-aurora-scheduler.md b/docs/deploying-aurora-scheduler.md
index c62354e..5f89ca9 100644
--- a/docs/deploying-aurora-scheduler.md
+++ b/docs/deploying-aurora-scheduler.md
@@ -164,20 +164,38 @@ wrapper script and executor are correctly copied into the sandbox. Finally, ensu
 script does not access resources outside of the sandbox, as when the script is run from within a
 docker container those resources will not exist.
 
+In order to correctly execute processes inside a job, the docker container must have python 2.7
+installed.
+
 A scheduler flag, `-global_container_mounts` allows mounting paths from the host (i.e., the slave)
 into all containers on that host. The format is a comma separated list of host_path:container_path[:mode]
 tuples. For example `-global_container_mounts=/opt/secret_keys_dir:/mnt/secret_keys_dir:ro` mounts
 `/opt/secret_keys_dir` from the slaves into all launched containers. Valid modes are `ro` and `rw`.
 
-In order to correctly execute processes inside a job, the docker container must have python 2.7
-installed.
+If you would like to supply your own parameters to `docker run` when launching jobs in docker
+containers, you may use the following flags:
+
+    -allow_docker_parameters
+    -default_docker_parameters
+
+`-allow_docker_parameters` controls whether or not users may pass their own configuration parameters
+through the job configuration files. If set to `false` (the default), the scheduler will reject
+jobs with custom parameters. *NOTE*: this setting should be used with caution as it allows any job
+owner to specify any parameters they wish, including those that may introduce security concerns
+(`privileged=true`, for example).
+
+`-default_docker_parameters` allows a cluster operator to specify a universal set of parameters that
+should be used for every container that does not have parameters explicitly configured at the job
+level. The argument accepts a multimap format:
+
+    -default_docker_parameters="read-only=true,tmpfs=/tmp,tmpfs=/run"
 
 ### Process Logs
 
 #### Log destination
 By default, Thermos will write process stdout/stderr to log files in the sandbox. Process object configuration
 allows specifying alternate log file destinations like streamed stdout/stderr or suppression of all log output.
-Default behavior can be configured for the entire cluster with the following flag (through the -thermos_executor_flags
+Default behavior can be configured for the entire cluster with the following flag (through the `-thermos_executor_flags`
 argument to the Aurora scheduler):
 
     --runner-logger-destination=both

http://git-wip-us.apache.org/repos/asf/aurora/blob/f0287252/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 a25fa41..a0d2a71 100644
--- a/src/main/java/org/apache/aurora/scheduler/app/AppModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/app/AppModule.java
@@ -17,7 +17,9 @@ import java.util.Set;
 
 import javax.inject.Singleton;
 
+import com.google.common.collect.ImmutableMultimap;
 import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Multimap;
 import com.google.inject.AbstractModule;
 
 import org.apache.aurora.GuiceUtils;
@@ -81,12 +83,18 @@ public class AppModule extends AbstractModule {
       help = "Allow to pass docker container parameters in the job.")
   private static final Arg<Boolean> ENABLE_DOCKER_PARAMETERS = Arg.create(false);
 
+  @CmdLine(name = "default_docker_parameters",
+      help = "Default docker parameters for any job that does not explicitly declare parameters.")
+  private static final Arg<Multimap<String, String>> DEFAULT_DOCKER_PARAMETERS =
+      Arg.create(ImmutableMultimap.of());
+
   @Override
   protected void configure() {
     bind(ConfigurationManager.class).toInstance(
         new ConfigurationManager(
             ImmutableSet.copyOf(ALLOWED_CONTAINER_TYPES.get()),
-            ENABLE_DOCKER_PARAMETERS.get()));
+            ENABLE_DOCKER_PARAMETERS.get(),
+            DEFAULT_DOCKER_PARAMETERS.get()));
     bind(Thresholds.class)
         .toInstance(new Thresholds(MAX_TASKS_PER_JOB.get(), MAX_UPDATE_INSTANCE_FAILURES.get()));
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/f0287252/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 7c249b3..376f485 100644
--- a/src/main/java/org/apache/aurora/scheduler/base/TaskTestUtil.java
+++ b/src/main/java/org/apache/aurora/scheduler/base/TaskTestUtil.java
@@ -15,6 +15,7 @@ package org.apache.aurora.scheduler.base;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableMultimap;
 import com.google.common.collect.ImmutableSet;
 
 import org.apache.aurora.gen.AssignedTask;
@@ -50,7 +51,7 @@ public final class TaskTestUtil {
   public static final IJobKey JOB = JobKeys.from("role", "env", "job");
   public static final TierInfo REVOCABLE_TIER = new TierInfo(true);
   public static final ConfigurationManager CONFIGURATION_MANAGER =
-      new ConfigurationManager(ImmutableSet.of(_Fields.MESOS), false);
+      new ConfigurationManager(ImmutableSet.of(_Fields.MESOS), false, ImmutableMultimap.of());
 
   private TaskTestUtil() {
     // Utility class.

http://git-wip-us.apache.org/repos/asf/aurora/blob/f0287252/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 6a5f9c5..e4dbf06 100644
--- a/src/main/java/org/apache/aurora/scheduler/configuration/ConfigurationManager.java
+++ b/src/main/java/org/apache/aurora/scheduler/configuration/ConfigurationManager.java
@@ -13,6 +13,7 @@
  */
 package org.apache.aurora.scheduler.configuration;
 
+import java.util.Map;
 import java.util.Objects;
 
 import javax.annotation.Nullable;
@@ -24,8 +25,10 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
+import com.google.common.collect.Multimap;
 
 import org.apache.aurora.gen.Container;
+import org.apache.aurora.gen.DockerParameter;
 import org.apache.aurora.gen.JobConfiguration;
 import org.apache.aurora.gen.TaskConfig;
 import org.apache.aurora.gen.TaskConfig._Fields;
@@ -98,13 +101,16 @@ public class ConfigurationManager {
 
   private final ImmutableSet<Container._Fields> allowedContainerTypes;
   private final boolean allowDockerParameters;
+  private final Multimap<String, String> defaultDockerParameters;
 
   public ConfigurationManager(
       ImmutableSet<Container._Fields> allowedContainerTypes,
-      boolean allowDockerParameters) {
+      boolean allowDockerParameters,
+      Multimap<String, String> defaultDockerParameters) {
 
     this.allowedContainerTypes = Objects.requireNonNull(allowedContainerTypes);
     this.allowDockerParameters = allowDockerParameters;
+    this.defaultDockerParameters = Objects.requireNonNull(defaultDockerParameters);
   }
 
   private static void requireNonNull(Object value, String error) throws TaskDescriptionException {
@@ -282,12 +288,18 @@ public class ConfigurationManager {
       containerType = Optional.of(containerConfig.getSetField());
       if (containerConfig.isSetDocker()) {
         if (!containerConfig.getDocker().isSetImage()) {
-          throw new TaskDescriptionException("A container must specify an image");
+          throw new TaskDescriptionException("A container must specify an image.");
         }
-        if (containerConfig.getDocker().isSetParameters()
-            && !containerConfig.getDocker().getParameters().isEmpty()
-            && !allowDockerParameters) {
-          throw new TaskDescriptionException("Docker parameters not allowed.");
+        if (!containerConfig.getDocker().isSetParameters()
+            || containerConfig.getDocker().getParameters().isEmpty()) {
+          for (Map.Entry<String, String> e : this.defaultDockerParameters.entries()) {
+            builder.getContainer().getDocker().addToParameters(
+                new DockerParameter(e.getKey(), e.getValue()));
+          }
+        } else {
+          if (!allowDockerParameters) {
+            throw new TaskDescriptionException("Docker parameters not allowed.");
+          }
         }
       }
     } else {

http://git-wip-us.apache.org/repos/asf/aurora/blob/f0287252/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 4286795..317506e 100644
--- a/src/test/java/org/apache/aurora/scheduler/configuration/ConfigurationManagerTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/configuration/ConfigurationManagerTest.java
@@ -13,13 +13,18 @@
  */
 package org.apache.aurora.scheduler.configuration;
 
+import java.util.Arrays;
+import java.util.List;
+
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableMultimap;
 import com.google.common.collect.ImmutableSet;
 
 import org.apache.aurora.gen.Constraint;
 import org.apache.aurora.gen.Container;
 import org.apache.aurora.gen.CronCollisionPolicy;
 import org.apache.aurora.gen.DockerContainer;
+import org.apache.aurora.gen.DockerParameter;
 import org.apache.aurora.gen.ExecutorConfig;
 import org.apache.aurora.gen.Identity;
 import org.apache.aurora.gen.JobConfiguration;
@@ -29,19 +34,31 @@ 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.IDockerParameter;
 import org.apache.aurora.scheduler.storage.entities.ITaskConfig;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.ExpectedException;
 
 import static org.apache.aurora.gen.test.testConstants.INVALID_IDENTIFIERS;
 import static org.apache.aurora.gen.test.testConstants.VALID_IDENTIFIERS;
 import static org.apache.aurora.scheduler.base.UserProvidedStrings.isGoodIdentifier;
 import static org.apache.aurora.scheduler.configuration.ConfigurationManager.DEDICATED_ATTRIBUTE;
+import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
 // TODO(kevints): Improve test coverage for this class.
 public class ConfigurationManagerTest {
+
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  private static final ImmutableSet<Container._Fields> ALL_CONTAINER_TYPES =
+      ImmutableSet.of(Container._Fields.DOCKER, Container._Fields.MESOS);
+
   private static final JobConfiguration UNSANITIZED_JOB_CONFIGURATION = new JobConfiguration()
       .setKey(new JobKey("owner-role", "devel", "email_stats"))
       .setCronSchedule("0 2 * * *")
@@ -95,10 +112,15 @@ public class ConfigurationManagerTest {
       .newBuilder();
 
   private ConfigurationManager configurationManager;
+  private ConfigurationManager dockerConfigurationManager;
 
   @Before
   public void setUp() {
-    configurationManager = new ConfigurationManager(ImmutableSet.of(), false);
+    configurationManager = new ConfigurationManager(
+        ALL_CONTAINER_TYPES, false, ImmutableMultimap.of());
+
+    dockerConfigurationManager = new ConfigurationManager(
+        ALL_CONTAINER_TYPES, true, ImmutableMultimap.of("foo", "bar"));
   }
 
   @Test
@@ -111,21 +133,67 @@ public class ConfigurationManagerTest {
     }
   }
 
-  @Test(expected = TaskDescriptionException.class)
+  @Test
   public void testBadContainerConfig() throws TaskDescriptionException {
     TaskConfig taskConfig = CONFIG_WITH_CONTAINER.deepCopy();
     taskConfig.getContainer().getDocker().setImage(null);
 
+    expectTaskDescriptionException("A container must specify an image");
     configurationManager.validateAndPopulate(ITaskConfig.build(taskConfig));
   }
 
-  @Test(expected = TaskDescriptionException.class)
+  @Test
+  public void testDisallowedDockerParameters() throws TaskDescriptionException {
+    TaskConfig taskConfig = CONFIG_WITH_CONTAINER.deepCopy();
+    taskConfig.getContainer().getDocker().addToParameters(new DockerParameter("foo", "bar"));
+
+    ConfigurationManager noParamsManager = new ConfigurationManager(
+        ALL_CONTAINER_TYPES, false, ImmutableMultimap.of());
+
+    expectTaskDescriptionException("Docker parameters not allowed");
+    noParamsManager.validateAndPopulate(ITaskConfig.build(taskConfig));
+  }
+
+  @Test
   public void testInvalidTier() throws TaskDescriptionException {
     ITaskConfig config = ITaskConfig.build(UNSANITIZED_JOB_CONFIGURATION.deepCopy().getTaskConfig()
         .setJobName("job")
         .setEnvironment("env")
         .setTier("pr/d"));
 
+    expectTaskDescriptionException("Tier contains illegal characters");
     configurationManager.validateAndPopulate(config);
   }
+
+  @Test
+  public void testDefaultDockerParameters() throws TaskDescriptionException {
+    ITaskConfig result = dockerConfigurationManager.validateAndPopulate(
+        ITaskConfig.build(CONFIG_WITH_CONTAINER.deepCopy()));
+
+    // The resulting task config should contain parameters supplied to the ConfigurationManager.
+    List<IDockerParameter> params = result.getContainer().getDocker().getParameters();
+    assertThat(
+        params, is(Arrays.asList(IDockerParameter.build(new DockerParameter("foo", "bar")))));
+  }
+
+  @Test
+  public void testPassthroughDockerParameters() throws TaskDescriptionException {
+    TaskConfig taskConfig = CONFIG_WITH_CONTAINER.deepCopy();
+    DockerParameter userParameter = new DockerParameter("bar", "baz");
+    taskConfig.getContainer().getDocker().getParameters().clear();
+    taskConfig.getContainer().getDocker().addToParameters(userParameter);
+
+    ITaskConfig result = dockerConfigurationManager.validateAndPopulate(
+        ITaskConfig.build(taskConfig));
+
+    // The resulting task config should contain parameters supplied from user config.
+    List<IDockerParameter> params = result.getContainer().getDocker().getParameters();
+    assertThat(
+        params, is(Arrays.asList(IDockerParameter.build(userParameter))));
+  }
+
+  private void expectTaskDescriptionException(String message) {
+    expectedException.expect(TaskDescriptionException.class);
+    expectedException.expectMessage(message);
+  }
 }