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/04/12 19:42:31 UTC

aurora git commit: Revert "Revert "Add support for storing and fetching images as properties of task configs.""

Repository: aurora
Updated Branches:
  refs/heads/master 71c9e674c -> a4853a43f


Revert "Revert "Add support for storing and fetching images as properties of task configs.""

This reverts commit b5c9e1bc46a623b5d898ec4dacbe132b79903dd7 and layers the changes needed to
support DB migrations on top as well.

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


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

Branch: refs/heads/master
Commit: a4853a43f6f9a4258fb69d91094a80a7803f8a8d
Parents: 71c9e67
Author: Joshua Cohen <jc...@apache.org>
Authored: Tue Apr 12 12:42:09 2016 -0500
Committer: Joshua Cohen <jc...@apache.org>
Committed: Tue Apr 12 12:42:09 2016 -0500

----------------------------------------------------------------------
 .../thrift/org/apache/aurora/gen/api.thrift     | 27 ++++++++++++
 config/checkstyle/suppressions.xml              |  4 +-
 docs/development/db-migration.md                | 13 +++---
 .../configuration/ConfigurationManager.java     |  9 ++++
 .../aurora/scheduler/storage/db/DbUtil.java     |  2 +-
 .../scheduler/storage/db/TaskConfigManager.java | 27 +++++++++++-
 .../scheduler/storage/db/TaskConfigMapper.java  | 24 ++++++++++
 .../migration/V001_CreateAppcImagesTable.java   | 46 ++++++++++++++++++++
 .../migration/V002_CreateDockerImagesTable.java | 46 ++++++++++++++++++++
 .../scheduler/storage/db/views/DbImage.java     | 38 ++++++++++++++++
 .../storage/db/views/DbTaskConfig.java          |  2 +
 .../scheduler/storage/db/TaskConfigMapper.xml   | 44 +++++++++++++++++++
 .../aurora/scheduler/storage/db/schema.sql      | 18 ++++++++
 .../configuration/ConfigurationManagerTest.java | 27 ++++++++++++
 .../storage/AbstractCronJobStoreTest.java       |  3 +-
 .../storage/AbstractTaskStoreTest.java          | 34 ++++++++++++++-
 .../storage/db/DbJobUpdateStoreTest.java        |  7 ++-
 17 files changed, 357 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/a4853a43/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 fc038d7..0cf4d6e 100644
--- a/api/src/main/thrift/org/apache/aurora/gen/api.thrift
+++ b/api/src/main/thrift/org/apache/aurora/gen/api.thrift
@@ -202,6 +202,28 @@ union Container {
   2: DockerContainer docker
 }
 
+/** Describes an image for use with the Mesos unified containerizer in the Docker format */
+struct DockerImage {
+  /** The name of the image to run */
+  1: string name
+  /** The Docker tag identifying the image */
+  2: string tag
+}
+
+/** Describes an image for use with the Mesos unified containerizer in the AppC format */
+struct AppcImage {
+  /** The name of the image to run */
+  1: string name
+  /** The appc image id identifying the image */
+  2: string imageId
+}
+
+/** Describes an image to be used with the Mesos unified containerizer */
+union Image {
+  1: DockerImage docker
+  2: AppcImage appc
+}
+
 /** Description of the tasks contained within a job. */
 struct TaskConfig {
  /** Job task belongs to. */
@@ -219,6 +241,11 @@ struct TaskConfig {
  18: optional bool production
  /** Task tier type. */
  30: optional string tier
+ /**
+  * If using the Mesos unified containerizer, the image to run (N.B. mutually exlusive with
+  * specifying a container)
+  */
+ 31: optional Image image
 
  20: set<Constraint> constraints
  /** a list of named ports this task requests */

http://git-wip-us.apache.org/repos/asf/aurora/blob/a4853a43/config/checkstyle/suppressions.xml
----------------------------------------------------------------------
diff --git a/config/checkstyle/suppressions.xml b/config/checkstyle/suppressions.xml
index d9fe92a..cd7fd0a 100644
--- a/config/checkstyle/suppressions.xml
+++ b/config/checkstyle/suppressions.xml
@@ -21,6 +21,6 @@ limitations under the License.
   <!-- Allow use of System.exit() in main. -->
   <suppress files="org/apache/aurora/scheduler/app/SchedulerMain.java"
             checks="RegexpSinglelineJava"/>
-  <suppress files="org/apache/aurora/scheduler/storage/db/testmigration/.*"
-            checks="TypeName" />
+  <suppress files="org/apache/aurora/scheduler/storage/db/migration/.*" checks="TypeName" />
+  <suppress files="org/apache/aurora/scheduler/storage/db/testmigration/.*" checks="TypeName" />
 </suppressions>

http://git-wip-us.apache.org/repos/asf/aurora/blob/a4853a43/docs/development/db-migration.md
----------------------------------------------------------------------
diff --git a/docs/development/db-migration.md b/docs/development/db-migration.md
index bddfee1..ad31b17 100644
--- a/docs/development/db-migration.md
+++ b/docs/development/db-migration.md
@@ -10,9 +10,10 @@ a snapshot is restored, no manual interaction is required by cluster operators.
 
 Upgrades
 --------
-When adding or altering tables or changing data, a new migration class should be created under the
-org.apache.aurora.scheduler.storage.db.migration package. The class should implement the
-[MigrationScript](https://github.com/mybatis/migrations/blob/master/src/main/java/org/apache/ibatis/migration/MigrationScript.java)
+When adding or altering tables or changing data, in addition to making to change in
+[schema.sql](../../src/main/resources/org/apache/aurora/scheduler/storage/db/schema.sql), a new
+migration class should be created under the org.apache.aurora.scheduler.storage.db.migration
+package. The class should implement the [MigrationScript](https://github.com/mybatis/migrations/blob/master/src/main/java/org/apache/ibatis/migration/MigrationScript.java)
 interface (see [V001_TestMigration](../../src/test/java/org/apache/aurora/scheduler/storage/db/testmigration/V001_TestMigration.java)
 as an example). The upgrade and downgrade scripts are defined in this class. When restoring a
 snapshot the list of migrations on the classpath is compared to the list of applied changes in the
@@ -28,6 +29,6 @@ applied.
 Baselines
 ---------
 After enough time has passed (at least 1 official release), it should be safe to baseline migrations
-if desired. This can be accomplished by adding the changes from migrations directly to
-[schema.sql](../../src/main/resources/org/apache/aurora/scheduler/storage/db/schema.sql), removing
-the corresponding migration classes and adding a migration to remove the changelog entries.
\ No newline at end of file
+if desired. This can be accomplished by ensuring the changes from migrations have been applied to
+[schema.sql](../../src/main/resources/org/apache/aurora/scheduler/storage/db/schema.sql) and then
+removing the corresponding migration classes and adding a migration to remove the changelog entries.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/aurora/blob/a4853a43/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 68573ba..041cec3 100644
--- a/src/main/java/org/apache/aurora/scheduler/configuration/ConfigurationManager.java
+++ b/src/main/java/org/apache/aurora/scheduler/configuration/ConfigurationManager.java
@@ -200,6 +200,10 @@ 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 CONTAINER_AND_IMAGE_ARE_MUTUALLY_EXCLUSIVE =
+      "A task may not have both a Docker container and an image.";
+
   /**
    * 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
@@ -292,6 +296,7 @@ public class ConfigurationManager {
       // Default to mesos container type if unset.
       containerType = Optional.of(Container._Fields.MESOS);
     }
+
     if (!containerType.isPresent()) {
       throw new TaskDescriptionException("A job must have a container type.");
     }
@@ -301,6 +306,10 @@ public class ConfigurationManager {
               + containerType.get().toString());
     }
 
+    if (containerType.get() != Container._Fields.MESOS && config.isSetImage()) {
+      throw new TaskDescriptionException(CONTAINER_AND_IMAGE_ARE_MUTUALLY_EXCLUSIVE);
+    }
+
     return ITaskConfig.build(builder);
   }
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/a4853a43/src/main/java/org/apache/aurora/scheduler/storage/db/DbUtil.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/DbUtil.java b/src/main/java/org/apache/aurora/scheduler/storage/db/DbUtil.java
index 7c1127e..942d180 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/db/DbUtil.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/DbUtil.java
@@ -79,7 +79,7 @@ public final class DbUtil {
   /**
    * Creates a new, empty storage system with a task store defined by the command line flag.
    *
-   * @return An new storage instance.
+   * @return A new storage instance.
    */
   public static Storage createFlaggedStorage() {
     return createStorageInjector(testModuleWithWorkQueue(PLAIN, Optional.absent()))

http://git-wip-us.apache.org/repos/asf/aurora/blob/a4853a43/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 364026a..25160df 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
@@ -22,8 +22,11 @@ import com.google.common.collect.Maps;
 
 import org.apache.aurora.scheduler.storage.db.views.DbTaskConfig;
 import org.apache.aurora.scheduler.storage.db.views.Pairs;
+import org.apache.aurora.scheduler.storage.entities.IAppcImage;
 import org.apache.aurora.scheduler.storage.entities.IConstraint;
 import org.apache.aurora.scheduler.storage.entities.IDockerContainer;
+import org.apache.aurora.scheduler.storage.entities.IDockerImage;
+import org.apache.aurora.scheduler.storage.entities.IImage;
 import org.apache.aurora.scheduler.storage.entities.ITaskConfig;
 import org.apache.aurora.scheduler.storage.entities.IValueConstraint;
 
@@ -107,7 +110,6 @@ class TaskConfigManager {
       configMapper.insertMetadata(configInsert.getId(), config.getMetadata());
     }
 
-    // TODO(wfarner): It would be nice if this generalized to different Container types.
     if (config.getContainer().isSetDocker()) {
       IDockerContainer container = config.getContainer().getDocker();
       InsertResult containerInsert = new InsertResult();
@@ -117,6 +119,29 @@ class TaskConfigManager {
       }
     }
 
+    if (config.isSetImage()) {
+      IImage image = config.getImage();
+
+      switch (image.getSetField()) {
+        case DOCKER:
+          IDockerImage dockerImage = image.getDocker();
+          configMapper.insertDockerImage(
+              configInsert.getId(),
+              dockerImage.getName(),
+              dockerImage.getTag());
+          break;
+        case APPC:
+          IAppcImage appcImage = image.getAppc();
+          configMapper.insertAppcImage(
+              configInsert.getId(),
+              appcImage.getName(),
+              appcImage.getImageId());
+          break;
+        default:
+          throw new IllegalStateException("Unexpected image type: " + image.getSetField());
+      }
+    }
+
     return configInsert.getId();
   }
 }

http://git-wip-us.apache.org/repos/asf/aurora/blob/a4853a43/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 12ca16b..e778a39 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
@@ -152,4 +152,28 @@ interface TaskConfigMapper extends GarbageCollectedTableMapper {
    * @param configIds Configs to delete.
    */
   void delete(@Param("configIds") Set<Long> configIds);
+
+  /**
+   * Inserts an AppC image association with an {@link ITaskConfig}.
+   *
+   * @param configId Task config ID.
+   * @param name The name of the image.
+   * @param imageId The image's identifier.
+   */
+  void insertAppcImage(
+      @Param("configId") long configId,
+      @Param("name") String name,
+      @Param("imageId") String imageId);
+
+  /**
+   * Inserts a Docker image association with an {@link ITaskConfig}.
+   *
+   * @param configId Task config ID.
+   * @param name The name of the image.
+   * @param tag The image's tag.
+   */
+  void insertDockerImage(
+      @Param("configId") long configId,
+      @Param("name") String name,
+      @Param("tag") String tag);
 }

http://git-wip-us.apache.org/repos/asf/aurora/blob/a4853a43/src/main/java/org/apache/aurora/scheduler/storage/db/migration/V001_CreateAppcImagesTable.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/migration/V001_CreateAppcImagesTable.java b/src/main/java/org/apache/aurora/scheduler/storage/db/migration/V001_CreateAppcImagesTable.java
new file mode 100644
index 0000000..0735772
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/migration/V001_CreateAppcImagesTable.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 V001_CreateAppcImagesTable implements MigrationScript {
+  @Override
+  public BigDecimal getId() {
+    return BigDecimal.valueOf(1L);
+  }
+
+  @Override
+  public String getDescription() {
+    return "Create the task_config_appc_images table.";
+  }
+
+  @Override
+  public String getUpScript() {
+    return "CREATE TABLE IF NOT EXISTS task_config_appc_images("
+        + "id IDENTITY,"
+        + "task_config_id BIGINT NOT NULL REFERENCES task_configs(id) ON DELETE CASCADE,"
+        + "name VARCHAR NOT NULL,"
+        + "image_id VARCHAR NOT NULL,"
+        + "UNIQUE(task_config_id)"
+        + ");";
+  }
+
+  @Override
+  public String getDownScript() {
+    return "DROP TABLE IF EXISTS task_config_appc_images;";
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/a4853a43/src/main/java/org/apache/aurora/scheduler/storage/db/migration/V002_CreateDockerImagesTable.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/migration/V002_CreateDockerImagesTable.java b/src/main/java/org/apache/aurora/scheduler/storage/db/migration/V002_CreateDockerImagesTable.java
new file mode 100644
index 0000000..9a1ef28
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/migration/V002_CreateDockerImagesTable.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 V002_CreateDockerImagesTable implements MigrationScript {
+  @Override
+  public BigDecimal getId() {
+    return BigDecimal.valueOf(2L);
+  }
+
+  @Override
+  public String getDescription() {
+    return "Create the task_config_docker_images table.";
+  }
+
+  @Override
+  public String getUpScript() {
+    return "CREATE TABLE IF NOT EXISTS task_config_docker_images("
+        + "id IDENTITY,"
+        + "task_config_id BIGINT NOT NULL REFERENCES task_configs(id) ON DELETE CASCADE,"
+        + "name VARCHAR NOT NULL,"
+        + "tag VARCHAR NOT NULL,"
+        + "UNIQUE(task_config_id)"
+        + ");";
+  }
+
+  @Override
+  public String getDownScript() {
+    return "DROP TABLE IF EXISTS task_config_docker_images;";
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/a4853a43/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbImage.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbImage.java b/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbImage.java
new file mode 100644
index 0000000..5964a2a
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbImage.java
@@ -0,0 +1,38 @@
+/**
+ * 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.views;
+
+import org.apache.aurora.gen.AppcImage;
+import org.apache.aurora.gen.DockerImage;
+import org.apache.aurora.gen.Image;
+
+public final class DbImage {
+  private AppcImage appc;
+  private DockerImage docker;
+
+  private DbImage() {
+  }
+
+  Image toThrift() {
+    if (appc != null) {
+      return Image.appc(appc);
+    }
+
+    if (docker != null) {
+      return Image.docker(docker);
+    }
+
+    throw new IllegalStateException("Unknown image type.");
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/a4853a43/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 eb848ad..cdd1060 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
@@ -47,6 +47,7 @@ public final class DbTaskConfig {
   private List<Metadata> metadata;
   private DbContainer container;
   private String tier;
+  private DbImage image;
 
   private DbTaskConfig() {
   }
@@ -67,6 +68,7 @@ public final class DbTaskConfig {
         .setMaxTaskFailures(maxTaskFailures)
         .setProduction(production)
         .setTier(tier)
+        .setImage(image == null ? null : image.toThrift())
         .setConstraints(constraints.stream()
             .map(DbConstraint::toThrift)
             .collect(GuavaUtils.toImmutableSet()))

http://git-wip-us.apache.org/repos/asf/aurora/blob/a4853a43/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 5218967..36df7ed 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
@@ -108,6 +108,19 @@
     <association property="docker" resultMap="dockerContainerMap"/>
   </resultMap>
 
+  <resultMap id="dockerImageMap" type="org.apache.aurora.gen.DockerImage">
+    <id column="id" />
+  </resultMap>
+
+  <resultMap id="appcImageMap" type="org.apache.aurora.gen.AppcImage">
+    <id column="id" />
+  </resultMap>
+
+  <resultMap id="imageMap" type="org.apache.aurora.scheduler.storage.db.views.DbImage">
+    <association property="appc" columnPrefix="appc_" resultMap="appcImageMap" />
+    <association property="docker" columnPrefix="docker_" resultMap="dockerImageMap" />
+  </resultMap>
+
   <resultMap id="metadataMap" type="org.apache.aurora.gen.Metadata">
     <id column="id" />
   </resultMap>
@@ -122,6 +135,7 @@
         resultMap="org.apache.aurora.scheduler.storage.db.JobKeyMapper.jobKeyMap"
         columnPrefix="j_"/>
     <association property="container" resultMap="containerMap" columnPrefix="c_"/>
+    <association property="image" resultMap="imageMap" columnPrefix="image_" />
     <collection
         property="constraints"
         columnPrefix="constraint_"
@@ -161,6 +175,10 @@
       m.id AS m_id,
       m.key AS m_key,
       m.value AS m_value,
+      di.name as image_docker_name,
+      di.tag as image_docker_tag,
+      ai.name as image_appc_name,
+      ai.image_id as image_appc_image_id,
       tc.id AS constraint_id,
       tc.name AS constraint_name,
       tlc.id AS constraint_l_id,
@@ -173,6 +191,8 @@
     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_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
     LEFT OUTER JOIN limit_constraints as tlc ON tlc.constraint_id = tc.id
     LEFT OUTER JOIN value_constraints as tvc ON tvc.constraint_id = tc.id
@@ -299,6 +319,30 @@
     )
   </insert>
 
+  <insert id="insertDockerImage">
+    INSERT INTO task_config_docker_images (
+      task_config_id,
+      name,
+      tag
+    ) VALUES (
+      #{configId},
+      #{name},
+      #{tag}
+    )
+  </insert>
+
+  <insert id="insertAppcImage">
+    INSERT INTO task_config_appc_images (
+      task_config_id,
+      name,
+      image_id
+    ) VALUES (
+      #{configId},
+      #{name},
+      #{imageId}
+    )
+  </insert>
+
   <insert id="insertMetadata">
     INSERT INTO task_config_metadata (
       task_config_id,

http://git-wip-us.apache.org/repos/asf/aurora/blob/a4853a43/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 be60c3b..92a0798 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
@@ -171,6 +171,24 @@ CREATE TABLE task_config_docker_container_parameters(
   value VARCHAR NOT NULL
 );
 
+CREATE TABLE task_config_docker_images(
+  id IDENTITY,
+  task_config_id BIGINT NOT NULL REFERENCES task_configs(id) ON DELETE CASCADE,
+  name VARCHAR NOT NULL,
+  tag VARCHAR NOT NULL,
+
+  UNIQUE(task_config_id)
+);
+
+CREATE TABLE task_config_appc_images(
+  id IDENTITY,
+  task_config_id BIGINT NOT NULL REFERENCES task_configs(id) ON DELETE CASCADE,
+  name VARCHAR NOT NULL,
+  image_id VARCHAR NOT NULL,
+
+  UNIQUE(task_config_id)
+);
+
 CREATE TABLE task_states(
   id INT PRIMARY KEY,
   name VARCHAR NOT NULL,

http://git-wip-us.apache.org/repos/asf/aurora/blob/a4853a43/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 ecd48ec..1ccfe01 100644
--- a/src/test/java/org/apache/aurora/scheduler/configuration/ConfigurationManagerTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/configuration/ConfigurationManagerTest.java
@@ -23,9 +23,11 @@ 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.DockerImage;
 import org.apache.aurora.gen.DockerParameter;
 import org.apache.aurora.gen.ExecutorConfig;
 import org.apache.aurora.gen.Identity;
+import org.apache.aurora.gen.Image;
 import org.apache.aurora.gen.JobConfiguration;
 import org.apache.aurora.gen.JobKey;
 import org.apache.aurora.gen.LimitConstraint;
@@ -233,6 +235,31 @@ public class ConfigurationManagerTest {
     CONFIGURATION_MANAGER.validateAndPopulate(ITaskConfig.build(builder));
   }
 
+  @Test
+  public void testImageAndDockerContainerConfigurationAreMutuallyExclusive() throws Exception {
+    TaskConfig builder = CONFIG_WITH_CONTAINER.newBuilder();
+    builder.getContainer().getDocker().unsetParameters();
+
+    Image image = new Image();
+    image.setDocker(new DockerImage().setName("my-container").setTag("tag"));
+    builder.setImage(image);
+
+    expectTaskDescriptionException(ConfigurationManager.CONTAINER_AND_IMAGE_ARE_MUTUALLY_EXCLUSIVE);
+    CONFIGURATION_MANAGER.validateAndPopulate(ITaskConfig.build(builder));
+  }
+
+  @Test
+  public void testImageWithoutContainerIsAllowed() throws Exception {
+    TaskConfig builder = UNSANITIZED_JOB_CONFIGURATION.deepCopy().getTaskConfig();
+    builder.unsetConstraints();
+
+    Image image = new Image();
+    image.setDocker(new DockerImage().setName("my-container").setTag("tag"));
+    builder.setImage(image);
+
+    CONFIGURATION_MANAGER.validateAndPopulate(ITaskConfig.build(builder));
+  }
+
   private void expectTaskDescriptionException(String message) {
     expectedException.expect(TaskDescriptionException.class);
     expectedException.expectMessage(message);

http://git-wip-us.apache.org/repos/asf/aurora/blob/a4853a43/src/test/java/org/apache/aurora/scheduler/storage/AbstractCronJobStoreTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/storage/AbstractCronJobStoreTest.java b/src/test/java/org/apache/aurora/scheduler/storage/AbstractCronJobStoreTest.java
index c316e49..2343394 100644
--- a/src/test/java/org/apache/aurora/scheduler/storage/AbstractCronJobStoreTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/storage/AbstractCronJobStoreTest.java
@@ -142,7 +142,8 @@ public abstract class AbstractCronJobStoreTest {
                 .setCronSchedule("schedule")
                 .setCronCollisionPolicy(CronCollisionPolicy.CANCEL_NEW)
                 .setTaskConfig(config.newBuilder())
-                .setInstanceCount(5)));
+                .setInstanceCount(5)),
+        StorageEntityUtil.getField(ITaskConfig.class, "image"));
   }
 
   private Set<IJobConfiguration> fetchJobs() {

http://git-wip-us.apache.org/repos/asf/aurora/blob/a4853a43/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 5895afa..310c4d8 100644
--- a/src/test/java/org/apache/aurora/scheduler/storage/AbstractTaskStoreTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/storage/AbstractTaskStoreTest.java
@@ -40,14 +40,18 @@ import com.google.inject.Injector;
 import com.google.inject.Module;
 
 import org.apache.aurora.common.testing.TearDownTestCase;
+import org.apache.aurora.gen.AppcImage;
 import org.apache.aurora.gen.Attribute;
 import org.apache.aurora.gen.Container;
+import org.apache.aurora.gen.DockerImage;
 import org.apache.aurora.gen.ExecutorConfig;
 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.Metadata;
 import org.apache.aurora.gen.ScheduledTask;
+import org.apache.aurora.gen.TaskConfig;
 import org.apache.aurora.gen.TaskQuery;
 import org.apache.aurora.scheduler.base.JobKeys;
 import org.apache.aurora.scheduler.base.Query;
@@ -149,7 +153,9 @@ public abstract class AbstractTaskStoreTest extends TearDownTestCase {
   @Test
   public void testSave() {
     IScheduledTask aWithHost = setHost(TASK_A, HOST_A);
-    StorageEntityUtil.assertFullyPopulated(aWithHost.newBuilder());
+    StorageEntityUtil.assertFullyPopulated(
+        aWithHost.newBuilder(),
+        StorageEntityUtil.getField(TaskConfig.class, "image"));
 
     saveTasks(aWithHost, TASK_B);
     assertStoreContents(aWithHost, TASK_B);
@@ -176,6 +182,32 @@ public abstract class AbstractTaskStoreTest extends TearDownTestCase {
   }
 
   @Test
+  public void testSaveWithDockerImage() {
+    ScheduledTask builder = TASK_A.newBuilder();
+
+    Image image = new Image();
+    image.setDocker(new DockerImage().setName("some-name").setTag("some-tag"));
+    builder.getAssignedTask().getTask().setImage(image);
+
+    IScheduledTask task = IScheduledTask.build(builder);
+    saveTasks(task);
+    assertStoreContents(task);
+  }
+
+  @Test
+  public void testSaveWithAppcImage() {
+    ScheduledTask builder = TASK_A.newBuilder();
+
+    Image image = new Image();
+    image.setAppc(new AppcImage().setName("some-name").setImageId("some-tag"));
+    builder.getAssignedTask().getTask().setImage(image);
+
+    IScheduledTask task = IScheduledTask.build(builder);
+    saveTasks(task);
+    assertStoreContents(task);
+  }
+
+  @Test
   public void testQuery() {
     assertStoreContents();
     saveTasks(TASK_A, TASK_B, TASK_C, TASK_D);

http://git-wip-us.apache.org/repos/asf/aurora/blob/a4853a43/src/test/java/org/apache/aurora/scheduler/storage/db/DbJobUpdateStoreTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/storage/db/DbJobUpdateStoreTest.java b/src/test/java/org/apache/aurora/scheduler/storage/db/DbJobUpdateStoreTest.java
index 0853039..e43ec6c 100644
--- a/src/test/java/org/apache/aurora/scheduler/storage/db/DbJobUpdateStoreTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/storage/db/DbJobUpdateStoreTest.java
@@ -60,6 +60,7 @@ import org.apache.aurora.scheduler.storage.entities.IJobUpdateKey;
 import org.apache.aurora.scheduler.storage.entities.IJobUpdateQuery;
 import org.apache.aurora.scheduler.storage.entities.IJobUpdateSummary;
 import org.apache.aurora.scheduler.storage.entities.ILock;
+import org.apache.aurora.scheduler.storage.entities.ITaskConfig;
 import org.apache.aurora.scheduler.storage.testing.StorageEntityUtil;
 import org.apache.aurora.scheduler.testing.FakeStatsProvider;
 import org.junit.After;
@@ -146,7 +147,8 @@ public class DbJobUpdateStoreTest {
         StorageEntityUtil.getField(JobUpdateSummary.class, "state"),
         StorageEntityUtil.getField(IJobUpdateSummary.class, "state"),
         StorageEntityUtil.getField(Range.class, "first"),
-        StorageEntityUtil.getField(Range.class, "last"));
+        StorageEntityUtil.getField(Range.class, "last"),
+        StorageEntityUtil.getField(ITaskConfig.class, "image"));
     saveUpdate(update1, Optional.of("lock1"));
     assertUpdate(update1);
 
@@ -182,7 +184,8 @@ public class DbJobUpdateStoreTest {
         StorageEntityUtil.getField(JobUpdateSummary.class, "state"),
         StorageEntityUtil.getField(IJobUpdateSummary.class, "state"),
         StorageEntityUtil.getField(Range.class, "first"),
-        StorageEntityUtil.getField(Range.class, "last"));
+        StorageEntityUtil.getField(Range.class, "last"),
+        StorageEntityUtil.getField(ITaskConfig.class, "image"));
     saveUpdate(update, Optional.of("lock1"));
     assertUpdate(update);
   }