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 2016/06/29 15:59:31 UTC

aurora git commit: Add support for Mesos Fetcher

Repository: aurora
Updated Branches:
  refs/heads/master 102f5f0a9 -> 4e28b9c8b


Add support for Mesos Fetcher

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


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

Branch: refs/heads/master
Commit: 4e28b9c8b29b66f2f10b0a6cafdec1f8e2c1bd7b
Parents: 102f5f0
Author: Renan DelValle <rd...@binghamton.edu>
Authored: Wed Jun 29 08:59:19 2016 -0700
Committer: Maxim Khutornenko <ma...@apache.org>
Committed: Wed Jun 29 08:59:19 2016 -0700

----------------------------------------------------------------------
 RELEASE-NOTES.md                                | 12 +++++
 .../thrift/org/apache/aurora/gen/api.thrift     | 12 +++++
 docs/features/mesos-fetcher.md                  | 46 ++++++++++++++++++++
 .../apache/aurora/scheduler/app/AppModule.java  |  8 +++-
 .../aurora/scheduler/base/TaskTestUtil.java     |  5 +++
 .../configuration/ConfigurationManager.java     | 12 ++++-
 .../scheduler/mesos/MesosTaskFactory.java       | 13 ++++++
 .../scheduler/storage/db/TaskConfigManager.java |  4 ++
 .../scheduler/storage/db/TaskConfigMapper.java  | 13 +++++-
 .../V007_CreateMesosFetcherURIsTable.java       | 46 ++++++++++++++++++++
 .../storage/db/views/DbTaskConfig.java          |  3 ++
 .../scheduler/storage/db/TaskConfigMapper.xml   | 26 +++++++++++
 .../aurora/scheduler/storage/db/schema.sql      |  8 ++++
 .../configuration/ConfigurationManagerTest.java | 25 +++++++++++
 .../mesos/MesosTaskFactoryImplTest.java         | 13 ++++++
 .../storage/AbstractTaskStoreTest.java          | 13 ++++++
 .../aurora/scheduler/thrift/Fixtures.java       |  1 +
 .../aurora/scheduler/thrift/ThriftIT.java       |  1 +
 18 files changed, 258 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/4e28b9c8/RELEASE-NOTES.md
----------------------------------------------------------------------
diff --git a/RELEASE-NOTES.md b/RELEASE-NOTES.md
index af2061c..5beda5d 100644
--- a/RELEASE-NOTES.md
+++ b/RELEASE-NOTES.md
@@ -1,3 +1,15 @@
+0.15.0
+======
+
+### New/updated:
+
+- New scheduler commandline argument -enable_mesos_fetcher to allow job submissions
+to contain URIs which will be passed to the Mesos Fetcher and subsequently downloaded into
+the sandbox. Please note that enabling job submissions to download resources from arbitrary
+URIs may have security implications.
+
+### Deprecations and removals:
+
 0.14.0
 ======
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/4e28b9c8/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 3e6daf4..1d66208 100644
--- a/api/src/main/thrift/org/apache/aurora/gen/api.thrift
+++ b/api/src/main/thrift/org/apache/aurora/gen/api.thrift
@@ -142,6 +142,16 @@ struct InstanceKey {
   2: i32 instanceId
 }
 
+/** URI which mirrors CommandInfo.URI in the Mesos Protobuf */
+struct MesosFetcherURI {
+  /** Where to get the resource from */
+  1: string value
+  /** Extract compressed archive after downloading */
+  2: optional bool extract
+  /** Cache value using Mesos Fetcher caching mechanism **/
+  3: optional bool cache
+}
+
 struct ExecutorConfig {
   /** Name identifying the Executor. */
   1: string name
@@ -253,6 +263,8 @@ struct TaskConfig {
  20: set<Constraint> constraints
  /** a list of named ports this task requests */
  21: set<string> requestedPorts
+ /** Resources to retrieve with Mesos Fetcher */
+ 33: optional set<MesosFetcherURI> mesosFetcherUris
  /**
   * Custom links to include when displaying this task on the scheduler dashboard. Keys are anchor
   * text, values are URLs. Wildcards are supported for dynamic link crafting based on host, ports,

http://git-wip-us.apache.org/repos/asf/aurora/blob/4e28b9c8/docs/features/mesos-fetcher.md
----------------------------------------------------------------------
diff --git a/docs/features/mesos-fetcher.md b/docs/features/mesos-fetcher.md
new file mode 100644
index 0000000..3df64c9
--- /dev/null
+++ b/docs/features/mesos-fetcher.md
@@ -0,0 +1,46 @@
+Mesos Fetcher
+=============
+
+Mesos has support for downloading resources into the sandbox through the
+use of the [Mesos Fetcher](http://mesos.apache.org/documentation/latest/fetcher/)
+
+Aurora supports passing URIs to the Mesos Fetcher dynamically by including
+a list of URIs in job submissions.
+
+How to use
+----------
+The scheduler flag `-enable_mesos_fetcher` must be set to true.
+
+Currently only the scheduler side of this feature has been implemented
+so a modification to the existing client, or a custom Thrift client are required
+to make use of this feature.
+
+If using a custom Thrift client, the list of URIs must be included in TaskConfig
+as the `mesosFetcherUris` field.
+
+Each Mesos Fetcher URI has the following data members:
+
+|Property | Description|
+|---------|------|
+|value (required)  |Path to the resource needed in the sandbox.|
+|extract (optional)|Extract files from packed or compressed archives into the sandbox.|
+|cache (optional) | Use caching mechanism provided by Mesos for resources.|
+
+Note that this structure is very similar to the one provided for downloading
+resources needed for a [custom executor](../operations/configuration.md).
+
+This is because both features use the Mesos fetcher to retrieve resources into
+the sandbox. However, one, the custom executor feature, has a static set of URIs
+set in the server side, and the other, the Mesos Fetcher feature, is a dynamic set
+of URIs set at the time of job submission.
+
+Security Implications
+---------------------
+There are security implications that must be taken into account when enabling this feature.
+**Enabling this feature may potentially enable any job submitting user to perform a privilege escalation.**
+
+Until a more through solution is created, one step that has been taken to mitigate this issue
+is to statically mark every user submitted URI as non-executable. This is in contrast to the set of URIs
+set in the custom executor feature which may mark any URI as executable.
+
+If the need arises to mark a downloaded URI as executable, please consider using the custom executor feature.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/aurora/blob/4e28b9c8/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 6c7c75a..e5bafa4 100644
--- a/src/main/java/org/apache/aurora/scheduler/app/AppModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/app/AppModule.java
@@ -97,6 +97,11 @@ public class AppModule extends AbstractModule {
   @CmdLine(name = "allow_gpu_resource", help = "Allow jobs to request Mesos GPU resource.")
   private static final Arg<Boolean> ALLOW_GPU_RESOURCE = Arg.create(false);
 
+  @CmdLine(name = "enable_mesos_fetcher", help = "Allow jobs to pass URIs "
+      + "to the Mesos Fetcher. Note that enabling this feature could pose "
+      + "a privilege escalation threat.")
+  private static final Arg<Boolean> ENABLE_MESOS_FETCHER = Arg.create(false);
+
   private final ConfigurationManagerSettings configurationManagerSettings;
 
   @VisibleForTesting
@@ -110,7 +115,8 @@ public class AppModule extends AbstractModule {
         ENABLE_DOCKER_PARAMETERS.get(),
         DEFAULT_DOCKER_PARAMETERS.get(),
         REQUIRE_DOCKER_USE_EXECUTOR.get(),
-        ALLOW_GPU_RESOURCE.get()));
+        ALLOW_GPU_RESOURCE.get(),
+        ENABLE_MESOS_FETCHER.get()));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/aurora/blob/4e28b9c8/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 8103163..c43e04a 100644
--- a/src/main/java/org/apache/aurora/scheduler/base/TaskTestUtil.java
+++ b/src/main/java/org/apache/aurora/scheduler/base/TaskTestUtil.java
@@ -30,6 +30,7 @@ import org.apache.aurora.gen.DockerParameter;
 import org.apache.aurora.gen.ExecutorConfig;
 import org.apache.aurora.gen.Identity;
 import org.apache.aurora.gen.LimitConstraint;
+import org.apache.aurora.gen.MesosFetcherURI;
 import org.apache.aurora.gen.Metadata;
 import org.apache.aurora.gen.Resource;
 import org.apache.aurora.gen.ScheduleStatus;
@@ -78,6 +79,7 @@ public final class TaskTestUtil {
           false,
           ImmutableMultimap.of(),
           true,
+          true,
           true);
   public static final ConfigurationManager CONFIGURATION_MANAGER =
       new ConfigurationManager(CONFIGURATION_MANAGER_SETTINGS, TIER_MANAGER, THRIFT_BACKFILL);
@@ -110,6 +112,9 @@ public final class TaskTestUtil {
         .setTaskLinks(ImmutableMap.of("http", "link", "admin", "otherLink"))
         .setContactEmail("foo@bar.com")
         .setMetadata(ImmutableSet.of(new Metadata("key", "value")))
+        .setMesosFetcherUris(ImmutableSet.of(
+            new MesosFetcherURI("pathA").setExtract(true).setCache(true),
+            new MesosFetcherURI("pathB").setExtract(true).setCache(true)))
         .setExecutorConfig(new ExecutorConfig("name", "config"))
         .setContainer(Container.docker(
             new DockerContainer("imagename")

http://git-wip-us.apache.org/repos/asf/aurora/blob/4e28b9c8/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 fe18c0f..93ed360 100644
--- a/src/main/java/org/apache/aurora/scheduler/configuration/ConfigurationManager.java
+++ b/src/main/java/org/apache/aurora/scheduler/configuration/ConfigurationManager.java
@@ -116,19 +116,22 @@ public class ConfigurationManager {
     private final Multimap<String, String> defaultDockerParameters;
     private final boolean requireDockerUseExecutor;
     private final boolean allowGpuResource;
+    private final boolean enableMesosFetcher;
 
     public ConfigurationManagerSettings(
         ImmutableSet<Container._Fields> allowedContainerTypes,
         boolean allowDockerParameters,
         Multimap<String, String> defaultDockerParameters,
         boolean requireDockerUseExecutor,
-        boolean allowGpuResource) {
+        boolean allowGpuResource,
+        boolean enableMesosFetcher) {
 
       this.allowedContainerTypes = requireNonNull(allowedContainerTypes);
       this.allowDockerParameters = allowDockerParameters;
       this.defaultDockerParameters = requireNonNull(defaultDockerParameters);
       this.requireDockerUseExecutor = requireDockerUseExecutor;
       this.allowGpuResource = allowGpuResource;
+      this.enableMesosFetcher = enableMesosFetcher;
     }
   }
 
@@ -217,6 +220,9 @@ public class ConfigurationManager {
   static final String EXECUTOR_REQUIRED_WITH_DOCKER =
       "This scheduler is configured to require an executor for Docker-based tasks.";
 
+  @VisibleForTesting
+  static final String MESOS_FETCHER_DISABLED =
+      "Mesos Fetcher for individual jobs is disabled in this cluster.";
   /**
    * Check validity of and populates defaults in a task configuration.  This will return a deep copy
    * of the provided task configuration with default configuration values applied, and configuration
@@ -339,6 +345,10 @@ public class ConfigurationManager {
       throw new TaskDescriptionException("GPU resource support is disabled in this cluster.");
     }
 
+    if (!settings.enableMesosFetcher && !config.getMesosFetcherUris().isEmpty()) {
+      throw new TaskDescriptionException(MESOS_FETCHER_DISABLED);
+    }
+
     maybeFillLinks(builder);
 
     return ITaskConfig.build(builder);

http://git-wip-us.apache.org/repos/asf/aurora/blob/4e28b9c8/src/main/java/org/apache/aurora/scheduler/mesos/MesosTaskFactory.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/mesos/MesosTaskFactory.java b/src/main/java/org/apache/aurora/scheduler/mesos/MesosTaskFactory.java
index 3b01801..cbbd6be 100644
--- a/src/main/java/org/apache/aurora/scheduler/mesos/MesosTaskFactory.java
+++ b/src/main/java/org/apache/aurora/scheduler/mesos/MesosTaskFactory.java
@@ -13,8 +13,10 @@
  */
 package org.apache.aurora.scheduler.mesos;
 
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 import javax.inject.Inject;
 
@@ -261,6 +263,17 @@ public interface MesosTaskFactory {
       ExecutorInfo.Builder builder = executorSettings.getExecutorConfig().getExecutor().toBuilder()
           .setExecutorId(getExecutorId(task.getTaskId()))
           .setSource(getInstanceSourceName(task.getTask(), task.getInstanceId()));
+
+      //TODO: (rdelvalle) add output_file when Aurora's Mesos dep is updated (MESOS-4735)
+      List<CommandInfo.URI> mesosFetcherUris = task.getTask().getMesosFetcherUris().stream()
+          .map(u -> Protos.CommandInfo.URI.newBuilder().setValue(u.getValue())
+              .setExecutable(false)
+              .setExtract(u.isExtract())
+              .setCache(u.isCache()).build())
+          .collect(Collectors.toList());
+
+      builder.setCommand(builder.getCommand().toBuilder().addAllUris(mesosFetcherUris));
+
       Iterable<Resource> executorResources = acceptedOffer.getExecutorResources();
       LOG.debug(
           "Setting executor resources to {}",

http://git-wip-us.apache.org/repos/asf/aurora/blob/4e28b9c8/src/main/java/org/apache/aurora/scheduler/storage/db/TaskConfigManager.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/TaskConfigManager.java b/src/main/java/org/apache/aurora/scheduler/storage/db/TaskConfigManager.java
index c761642..e137e57 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/db/TaskConfigManager.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/TaskConfigManager.java
@@ -114,6 +114,10 @@ class TaskConfigManager {
       configMapper.insertMetadata(configInsert.getId(), config.getMetadata());
     }
 
+    if (!config.getMesosFetcherUris().isEmpty()) {
+      configMapper.insertMesosFetcherUris(configInsert.getId(), config.getMesosFetcherUris());
+    }
+
     if (config.getContainer().isSetDocker()) {
       IDockerContainer container = config.getContainer().getDocker();
       InsertResult containerInsert = new InsertResult();

http://git-wip-us.apache.org/repos/asf/aurora/blob/4e28b9c8/src/main/java/org/apache/aurora/scheduler/storage/db/TaskConfigMapper.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/TaskConfigMapper.java b/src/main/java/org/apache/aurora/scheduler/storage/db/TaskConfigMapper.java
index 5712010..151306a 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/db/TaskConfigMapper.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/TaskConfigMapper.java
@@ -24,6 +24,7 @@ import org.apache.aurora.scheduler.storage.entities.IDockerContainer;
 import org.apache.aurora.scheduler.storage.entities.IDockerParameter;
 import org.apache.aurora.scheduler.storage.entities.IJobKey;
 import org.apache.aurora.scheduler.storage.entities.ILimitConstraint;
+import org.apache.aurora.scheduler.storage.entities.IMesosFetcherURI;
 import org.apache.aurora.scheduler.storage.entities.IMetadata;
 import org.apache.aurora.scheduler.storage.entities.ITaskConfig;
 import org.apache.aurora.scheduler.storage.entities.IValueConstraint;
@@ -126,7 +127,7 @@ interface TaskConfigMapper extends GarbageCollectedTableMapper {
       @Param("result") InsertResult result);
 
   /**
-   * Inserts docker parameters in associationw ith an {@link IDockerContainer}.
+   * Inserts docker parameters in association with an {@link IDockerContainer}.
    *
    * @param containerId Docker container row ID.
    * @param parameters Parameters to insert.
@@ -146,6 +147,16 @@ interface TaskConfigMapper extends GarbageCollectedTableMapper {
       @Param("metadata") Set<IMetadata> metadata);
 
   /**
+   * Inserts the Mesos Fetcher URIs in association with an {@link ITaskConfig}.
+   *
+   * @param configId Task config ID.
+   * @param uris Resources Mesos Fetcher should place in sandbox.
+   */
+  void insertMesosFetcherUris(
+      @Param("configId") long configId,
+      @Param("uris") Set<IMesosFetcherURI> uris);
+
+  /**
    * Deletes task configs.
    *
    * @param configIds Configs to delete.

http://git-wip-us.apache.org/repos/asf/aurora/blob/4e28b9c8/src/main/java/org/apache/aurora/scheduler/storage/db/migration/V007_CreateMesosFetcherURIsTable.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/migration/V007_CreateMesosFetcherURIsTable.java b/src/main/java/org/apache/aurora/scheduler/storage/db/migration/V007_CreateMesosFetcherURIsTable.java
new file mode 100644
index 0000000..d474e17
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/migration/V007_CreateMesosFetcherURIsTable.java
@@ -0,0 +1,46 @@
+/**
+ * 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.scheduler.storage.db.migration;
+
+import java.math.BigDecimal;
+
+import org.apache.ibatis.migration.MigrationScript;
+
+public class V007_CreateMesosFetcherURIsTable implements MigrationScript {
+  @Override
+  public BigDecimal getId() {
+    return BigDecimal.valueOf(7L);
+  }
+
+  @Override
+  public String getDescription() {
+    return "Create the task_config_mesos_fetcher_uris table.";
+  }
+
+  @Override
+  public String getUpScript() {
+    return "CREATE TABLE IF NOT EXISTS task_config_mesos_fetcher_uris("
+        + "id IDENTITY,"
+        + "task_config_id BIGINT NOT NULL REFERENCES task_configs(id) ON DELETE CASCADE,"
+        + "value VARCHAR NOT NULL,"
+        + "extract BOOLEAN NOT NULL,"
+        + "cache BOOLEAN NOT NULL"
+        + ");";
+  }
+
+  @Override
+  public String getDownScript() {
+    return "DROP TABLE IF EXISTS task_config_mesos_fetcher_uris;";
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/4e28b9c8/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 a90cb00..cf65a28 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
@@ -23,6 +23,7 @@ import org.apache.aurora.gen.ExecutorConfig;
 import org.apache.aurora.gen.Identity;
 import org.apache.aurora.gen.JobKey;
 import org.apache.aurora.gen.MesosContainer;
+import org.apache.aurora.gen.MesosFetcherURI;
 import org.apache.aurora.gen.Metadata;
 import org.apache.aurora.gen.TaskConfig;
 import org.apache.aurora.scheduler.storage.entities.ITaskConfig;
@@ -46,6 +47,7 @@ public final class DbTaskConfig {
   private String contactEmail;
   private ExecutorConfig executorConfig;
   private List<Metadata> metadata;
+  private List<MesosFetcherURI> mesosFetcherUris;
   private DbContainer container;
   private String tier;
   private List<DBResource> resources;
@@ -77,6 +79,7 @@ public final class DbTaskConfig {
         .setContactEmail(contactEmail)
         .setExecutorConfig(executorConfig)
         .setMetadata(ImmutableSet.copyOf(metadata))
+        .setMesosFetcherUris(ImmutableSet.copyOf(mesosFetcherUris))
         .setContainer(
             container == null ? Container.mesos(new MesosContainer()) : container.toThrift())
         .setResources(resources.stream().map(DBResource::toThrift).collect(toImmutableSet()));

http://git-wip-us.apache.org/repos/asf/aurora/blob/4e28b9c8/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 2c8af8b..3fce25f 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
@@ -126,6 +126,10 @@
     <id column="id" />
   </resultMap>
 
+  <resultMap id="mesosFetcherUrisMap" type="org.apache.aurora.gen.MesosFetcherURI">
+    <id column="id" />
+  </resultMap>
+
   <resultMap id="resourceMap" type="org.apache.aurora.scheduler.storage.db.views.DBResource">
     <id column="id" />
     <result property="type"
@@ -151,6 +155,7 @@
       <result column="port_name" />
     </collection>
     <collection property="metadata" resultMap="metadataMap" columnPrefix="m_"/>
+    <collection property="mesosFetcherUris" resultMap="mesosFetcherUrisMap" columnPrefix="u_"/>
     <collection
         property="taskLinks"
         select="selectTaskLinks"
@@ -188,6 +193,10 @@
       m.id AS m_id,
       m.key AS m_key,
       m.value AS m_value,
+      u.id AS u_id,
+      u.value AS u_value,
+      u.extract AS u_extract,
+      u.cache AS u_cache,
       di.name as c_image_docker_name,
       di.tag as c_image_docker_tag,
       ai.name as c_image_appc_name,
@@ -204,6 +213,7 @@
     LEFT OUTER JOIN task_config_requested_ports AS p ON p.task_config_id = c.id
     LEFT OUTER JOIN task_config_docker_containers AS d ON d.task_config_id = c.id
     LEFT OUTER JOIN task_config_metadata AS m ON m.task_config_id = c.id
+    LEFT OUTER JOIN task_config_mesos_fetcher_uris AS u ON u.task_config_id = c.id
     LEFT OUTER JOIN task_config_docker_images AS di ON di.task_config_id = c.id
     LEFT OUTER JOIN task_config_appc_images AS ai ON ai.task_config_id = c.id
     LEFT OUTER JOIN task_constraints AS tc ON tc.task_config_id = c.id
@@ -393,6 +403,22 @@
     )
   </insert>
 
+  <insert id="insertMesosFetcherUris">
+    INSERT INTO task_config_mesos_fetcher_uris (
+      task_config_id,
+      value,
+      extract,
+      cache
+    ) VALUES (
+    <foreach item="entry" collection="uris" separator="),(">
+      #{configId},
+      #{entry.value},
+      #{entry.extract},
+      #{entry.cache}
+    </foreach>
+    )
+  </insert>
+
   <select id="selectAllRowIds" resultType="long">
     SELECT id FROM task_configs
   </select>

http://git-wip-us.apache.org/repos/asf/aurora/blob/4e28b9c8/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 5069bed..a40830f 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
@@ -181,6 +181,14 @@ CREATE TABLE task_config_metadata(
   value VARCHAR NOT NULL
 );
 
+CREATE TABLE task_config_mesos_fetcher_uris(
+  id IDENTITY,
+  task_config_id BIGINT NOT NULL REFERENCES task_configs(id) ON DELETE CASCADE,
+  value VARCHAR NOT NULL,
+  extract BOOLEAN NOT NULL,
+  cache BOOLEAN NOT NULL
+);
+
 CREATE TABLE task_config_docker_containers(
   id IDENTITY,
   task_config_id BIGINT NOT NULL REFERENCES task_configs(id) ON DELETE CASCADE,

http://git-wip-us.apache.org/repos/asf/aurora/blob/4e28b9c8/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 2dff80b..1473270 100644
--- a/src/test/java/org/apache/aurora/scheduler/configuration/ConfigurationManagerTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/configuration/ConfigurationManagerTest.java
@@ -29,6 +29,7 @@ import org.apache.aurora.gen.Identity;
 import org.apache.aurora.gen.JobConfiguration;
 import org.apache.aurora.gen.JobKey;
 import org.apache.aurora.gen.LimitConstraint;
+import org.apache.aurora.gen.MesosFetcherURI;
 import org.apache.aurora.gen.TaskConfig;
 import org.apache.aurora.gen.TaskConstraint;
 import org.apache.aurora.gen.ValueConstraint;
@@ -118,6 +119,7 @@ public class ConfigurationManagerTest {
           false,
           ImmutableMultimap.of(),
           true,
+          false,
           false),
       TaskTestUtil.TIER_MANAGER,
       TaskTestUtil.THRIFT_BACKFILL);
@@ -127,6 +129,7 @@ public class ConfigurationManagerTest {
           true,
           ImmutableMultimap.of("foo", "bar"),
           false,
+          true,
           true),
       TaskTestUtil.TIER_MANAGER,
       TaskTestUtil.THRIFT_BACKFILL);
@@ -288,12 +291,34 @@ public class ConfigurationManagerTest {
             true,
             ImmutableMultimap.of("foo", "bar"),
             false,
+            false,
             false),
         TaskTestUtil.TIER_MANAGER,
         TaskTestUtil.THRIFT_BACKFILL).validateAndPopulate(ITaskConfig.build(builder));
   }
 
   @Test
+  public void testMesosFetcherDisabled() throws Exception {
+    TaskConfig builder = CONFIG_WITH_CONTAINER.newBuilder();
+    builder.setMesosFetcherUris(
+        ImmutableSet.of(
+            new MesosFetcherURI("pathA").setExtract(true).setCache(true),
+            new MesosFetcherURI("pathB").setExtract(true).setCache(true)));
+
+    expectTaskDescriptionException(ConfigurationManager.MESOS_FETCHER_DISABLED);
+    new ConfigurationManager(
+            new ConfigurationManagerSettings(
+                    ALL_CONTAINER_TYPES,
+                    true,
+                    ImmutableMultimap.of("foo", "bar"),
+                    false,
+                    false,
+                    false),
+            TaskTestUtil.TIER_MANAGER,
+            TaskTestUtil.THRIFT_BACKFILL).validateAndPopulate(ITaskConfig.build(builder));
+  }
+
+  @Test
   public void testTaskLinks() throws Exception {
     TaskConfig builder = CONFIG_WITH_CONTAINER.newBuilder();
     builder.addToResources(namedPort("health"));

http://git-wip-us.apache.org/repos/asf/aurora/blob/4e28b9c8/src/test/java/org/apache/aurora/scheduler/mesos/MesosTaskFactoryImplTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/mesos/MesosTaskFactoryImplTest.java b/src/test/java/org/apache/aurora/scheduler/mesos/MesosTaskFactoryImplTest.java
index 58785bf..500fd43 100644
--- a/src/test/java/org/apache/aurora/scheduler/mesos/MesosTaskFactoryImplTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/mesos/MesosTaskFactoryImplTest.java
@@ -45,6 +45,7 @@ import org.apache.aurora.scheduler.storage.entities.IJobKey;
 import org.apache.aurora.scheduler.storage.entities.IServerInfo;
 import org.apache.aurora.scheduler.storage.entities.ITaskConfig;
 import org.apache.mesos.Protos;
+import org.apache.mesos.Protos.CommandInfo.URI;
 import org.apache.mesos.Protos.ContainerInfo;
 import org.apache.mesos.Protos.ContainerInfo.DockerInfo;
 import org.apache.mesos.Protos.ContainerInfo.MesosInfo;
@@ -152,6 +153,18 @@ public class MesosTaskFactoryImplTest extends EasyMockTest {
         .setExecutorId(MesosTaskFactoryImpl.getExecutorId(task.getTaskId()))
         .setSource(
             MesosTaskFactoryImpl.getInstanceSourceName(task.getTask(), task.getInstanceId()))
+        .setCommand(executor.getCommand().toBuilder().addAllUris(
+            ImmutableSet.of(
+                URI.newBuilder()
+                    .setValue("pathA")
+                    .setExecutable(false)
+                    .setExtract(true)
+                    .setCache(true).build(),
+                URI.newBuilder()
+                    .setValue("pathB")
+                    .setExecutable(false)
+                    .setExtract(true)
+                    .setCache(true).build())))
         .build();
   }
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/4e28b9c8/src/test/java/org/apache/aurora/scheduler/storage/AbstractTaskStoreTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/storage/AbstractTaskStoreTest.java b/src/test/java/org/apache/aurora/scheduler/storage/AbstractTaskStoreTest.java
index b1593f6..3d07292 100644
--- a/src/test/java/org/apache/aurora/scheduler/storage/AbstractTaskStoreTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/storage/AbstractTaskStoreTest.java
@@ -49,6 +49,7 @@ import org.apache.aurora.gen.HostAttributes;
 import org.apache.aurora.gen.Image;
 import org.apache.aurora.gen.MaintenanceMode;
 import org.apache.aurora.gen.MesosContainer;
+import org.apache.aurora.gen.MesosFetcherURI;
 import org.apache.aurora.gen.Metadata;
 import org.apache.aurora.gen.ScheduledTask;
 import org.apache.aurora.gen.TaskQuery;
@@ -179,6 +180,18 @@ public abstract class AbstractTaskStoreTest extends TearDownTestCase {
   }
 
   @Test
+  public void testSaveWithMesosFetcherUris() {
+    ScheduledTask builder = TASK_A.newBuilder();
+    builder.getAssignedTask().getTask().setMesosFetcherUris(
+        ImmutableSet.of(
+            new MesosFetcherURI("pathA").setExtract(true).setCache(true),
+            new MesosFetcherURI("pathB").setExtract(true).setCache(true)));
+    IScheduledTask task = IScheduledTask.build(builder);
+    saveTasks(task);
+    assertStoreContents(task);
+  }
+
+  @Test
   public void testSaveWithDockerImage() {
     ScheduledTask builder = TASK_A.newBuilder();
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/4e28b9c8/src/test/java/org/apache/aurora/scheduler/thrift/Fixtures.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/thrift/Fixtures.java b/src/test/java/org/apache/aurora/scheduler/thrift/Fixtures.java
index a883b0e..a79b0f4 100644
--- a/src/test/java/org/apache/aurora/scheduler/thrift/Fixtures.java
+++ b/src/test/java/org/apache/aurora/scheduler/thrift/Fixtures.java
@@ -113,6 +113,7 @@ final class Fixtures {
         .setMaxTaskFailures(1)
         .setConstraints(ImmutableSet.of())
         .setMetadata(ImmutableSet.of())
+        .setMesosFetcherUris(ImmutableSet.of())
         .setContainer(Container.mesos(new MesosContainer()))
         .setResources(ImmutableSet.of(
             Resource.numCpus(1),

http://git-wip-us.apache.org/repos/asf/aurora/blob/4e28b9c8/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 ed69996..3caad32 100644
--- a/src/test/java/org/apache/aurora/scheduler/thrift/ThriftIT.java
+++ b/src/test/java/org/apache/aurora/scheduler/thrift/ThriftIT.java
@@ -155,6 +155,7 @@ public class ThriftIT extends EasyMockTest {
         true,
         ImmutableMultimap.of(),
         false,
+        true,
         true);
 
     createThrift(configurationManagerSettings);