You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by zm...@apache.org on 2017/04/07 12:18:45 UTC

aurora git commit: Fix schema to allow multiple task volumes per task.

Repository: aurora
Updated Branches:
  refs/heads/master 656cf9ac5 -> cc2aa46f7


Fix schema to allow multiple task volumes per task.

The original commit adding this feature added an artifical constraint to the
schema that prevented more than one task volume per task. This is because there
was a `UNIQUE` constraint between the volumes table and the task config table,
preventing a task config from being associated with more than one volume.

This patch removes that constraint. As a result some of the MyBatis mappers had
to change and a new migration was added.

Bugs closed: AURORA-1914

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


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

Branch: refs/heads/master
Commit: cc2aa46f7ad8590e201621ffe2799299959ef7eb
Parents: 656cf9a
Author: Zameer Manji <zm...@apache.org>
Authored: Fri Apr 7 14:18:33 2017 +0200
Committer: Zameer Manji <zm...@apache.org>
Committed: Fri Apr 7 14:18:33 2017 +0200

----------------------------------------------------------------------
 .../migration/V010_RemoveUniqueConstraint.java  | 41 ++++++++++++++++++++
 .../scheduler/storage/db/views/DbContainer.java |  6 +--
 .../storage/db/views/DbTaskConfig.java          | 12 +++++-
 .../scheduler/storage/db/TaskConfigMapper.xml   | 14 ++++---
 .../aurora/scheduler/storage/db/schema.sql      |  2 -
 .../storage/AbstractTaskStoreTest.java          |  7 +++-
 .../apache/aurora/e2e/http/http_example.aurora  |  5 ++-
 7 files changed, 71 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/cc2aa46f/src/main/java/org/apache/aurora/scheduler/storage/db/migration/V010_RemoveUniqueConstraint.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/migration/V010_RemoveUniqueConstraint.java b/src/main/java/org/apache/aurora/scheduler/storage/db/migration/V010_RemoveUniqueConstraint.java
new file mode 100644
index 0000000..08c38f3
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/migration/V010_RemoveUniqueConstraint.java
@@ -0,0 +1,41 @@
+/**
+ * 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 V010_RemoveUniqueConstraint implements MigrationScript {
+  @Override
+  public BigDecimal getId() {
+    return BigDecimal.valueOf(10L);
+  }
+
+  @Override
+  public String getDescription() {
+    return "Remove unique constraint in task_config_volumes";
+  }
+
+  @Override
+  public String getUpScript() {
+    // The constraint name is taken from the schema so it is always constant.
+    return "ALTER TABLE IF EXISTS task_config_volumes DROP CONSTRAINT IF EXISTS CONSTRAINT_654B;";
+  }
+
+  @Override
+  public String getDownScript() {
+    return "ALTER TABLE IF EXISTS task_config_volumes ADD UNIQUE(task_config_id);";
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/cc2aa46f/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbContainer.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbContainer.java b/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbContainer.java
index 6a36e27..8d4d7ec 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbContainer.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbContainer.java
@@ -13,17 +13,13 @@
  */
 package org.apache.aurora.scheduler.storage.db.views;
 
-import java.util.List;
-
 import org.apache.aurora.gen.Container;
 import org.apache.aurora.gen.DockerContainer;
 import org.apache.aurora.gen.MesosContainer;
-import org.apache.aurora.gen.Volume;
 
 public final class DbContainer {
   private DockerContainer docker;
   private DbImage image;
-  private List<Volume> volumes;
 
   private DbContainer() {
   }
@@ -34,7 +30,7 @@ public final class DbContainer {
     }
 
     if (image != null) {
-      return Container.mesos(new MesosContainer().setImage(image.toThrift()).setVolumes(volumes));
+      return Container.mesos(new MesosContainer().setImage(image.toThrift()));
     }
 
     return Container.mesos(new MesosContainer());

http://git-wip-us.apache.org/repos/asf/aurora/blob/cc2aa46f/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 cf65a28..138cd53 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
@@ -26,6 +26,7 @@ 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.gen.Volume;
 import org.apache.aurora.scheduler.storage.entities.ITaskConfig;
 
 import static org.apache.aurora.GuavaUtils.toImmutableSet;
@@ -49,6 +50,7 @@ public final class DbTaskConfig {
   private List<Metadata> metadata;
   private List<MesosFetcherURI> mesosFetcherUris;
   private DbContainer container;
+  private List<Volume> volumes;
   private String tier;
   private List<DBResource> resources;
 
@@ -60,7 +62,7 @@ public final class DbTaskConfig {
   }
 
   TaskConfig toThrift() {
-    return new TaskConfig()
+    TaskConfig builder = new TaskConfig()
         .setJob(job)
         .setOwner(owner)
         .setIsService(isService)
@@ -83,6 +85,14 @@ public final class DbTaskConfig {
         .setContainer(
             container == null ? Container.mesos(new MesosContainer()) : container.toThrift())
         .setResources(resources.stream().map(DBResource::toThrift).collect(toImmutableSet()));
+
+    // In the DB Layer volumes are associated with a task config, since containers are not
+    // modelled as tables.
+    if (builder.getContainer().isSetMesos()) {
+      builder.getContainer().getMesos().setVolumes(volumes);
+    }
+
+    return builder;
   }
 
   public ITaskConfig toImmutable() {

http://git-wip-us.apache.org/repos/asf/aurora/blob/cc2aa46f/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 97f6b80..5422183 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
@@ -118,9 +118,12 @@
   </resultMap>
 
   <resultMap id="containerMap" type="org.apache.aurora.scheduler.storage.db.views.DbContainer">
+    <!--NOTE: Do not put any collections under here. ContainerMap doesn't correspond to a table, it's a syntheic map
+    for assocations. Since it doesn't have a table, there is no unique id. Since there is no unique id, MyBatis nested
+    result map collections fail.
+    -->
     <association property="docker" resultMap="dockerContainerMap" />
     <association property="image" columnPrefix="image_" resultMap="imageMap" />
-    <collection property="volumes" columnPrefix="volume_" resultMap="volumeMap" notNullColumn="id" />
   </resultMap>
 
   <resultMap id="metadataMap" type="org.apache.aurora.gen.Metadata">
@@ -153,6 +156,7 @@
         resultMap="org.apache.aurora.scheduler.storage.db.JobKeyMapper.jobKeyMap"
         columnPrefix="j_"/>
     <association property="container" resultMap="containerMap" columnPrefix="c_"/>
+    <collection property="volumes" columnPrefix="volume_" resultMap="volumeMap" notNullColumn="id" />
     <collection
         property="constraints"
         columnPrefix="constraint_"
@@ -207,10 +211,10 @@
       di.tag as c_image_docker_tag,
       ai.name as c_image_appc_name,
       ai.image_id as c_image_appc_image_id,
-      v.id as c_volume_id,
-      v.host_path as c_volume_host_path,
-      v.container_path as c_volume_container_path,
-      v.mode as c_volume_mode,
+      v.id as volume_id,
+      v.host_path as volume_host_path,
+      v.container_path as volume_container_path,
+      v.mode as volume_mode,
       tc.id AS constraint_id,
       tc.name AS constraint_name,
       tlc.id AS constraint_l_id,

http://git-wip-us.apache.org/repos/asf/aurora/blob/cc2aa46f/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 ff1623a..7a86f47 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
@@ -235,8 +235,6 @@ CREATE TABLE task_config_volumes(
   host_path VARCHAR NOT NULL,
   container_path VARCHAR NOT NULL,
   mode INT NOT NULL REFERENCES volume_modes(id),
-
-  UNIQUE(task_config_id)
 );
 
 CREATE TABLE task_states(

http://git-wip-us.apache.org/repos/asf/aurora/blob/cc2aa46f/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 80be339..69c35b1 100644
--- a/src/test/java/org/apache/aurora/scheduler/storage/AbstractTaskStoreTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/storage/AbstractTaskStoreTest.java
@@ -197,7 +197,9 @@ public abstract class AbstractTaskStoreTest extends TearDownTestCase {
   public void testSaveWithContainerVolumes() {
     ScheduledTask builder = TASK_B.newBuilder();
     Image image = Image.docker(new DockerImage().setName("some-name").setTag("some-tag"));
-    List<Volume> volumes = ImmutableList.of(new Volume("container", "host", Mode.RO));
+    List<Volume> volumes = ImmutableList.of(
+        new Volume("container2", "host2", Mode.RW),
+        new Volume("container", "host", Mode.RO));
     builder.getAssignedTask().getTask().getContainer().getMesos().setImage(image)
         .setVolumes(volumes);
 
@@ -683,7 +685,8 @@ public abstract class AbstractTaskStoreTest extends TearDownTestCase {
   private void assertQueryResults(Query.Builder query, Set<IScheduledTask> tasks) {
     Iterable<IScheduledTask> result = fetchTasks(query);
     assertQueryHasNoDupes(result);
-    assertEquals(tasks, ImmutableSet.copyOf(fetchTasks(query)));
+    Set<IScheduledTask> set = ImmutableSet.copyOf(result);
+    assertEquals(tasks, set);
   }
 
   private void assertQueryHasNoDupes(Iterable<IScheduledTask> result) {

http://git-wip-us.apache.org/repos/asf/aurora/blob/cc2aa46f/src/test/sh/org/apache/aurora/e2e/http/http_example.aurora
----------------------------------------------------------------------
diff --git a/src/test/sh/org/apache/aurora/e2e/http/http_example.aurora b/src/test/sh/org/apache/aurora/e2e/http/http_example.aurora
index be43a09..0361b36 100644
--- a/src/test/sh/org/apache/aurora/e2e/http/http_example.aurora
+++ b/src/test/sh/org/apache/aurora/e2e/http/http_example.aurora
@@ -127,7 +127,10 @@ job = Service(
   ),
 )
 
-volumes = [Volume(container_path="etc/rsyslog.d.container", host_path="/etc/rsyslog.d", mode=Mode("RO"))]
+volumes = [
+  Volume(container_path="etc/rsyslog.d.container", host_path="/etc/rsyslog.d", mode=Mode("RO")),
+  Volume(container_path="etc/w3m.container", host_path="/etc/w3m", mode=Mode("RO"))
+]
 
 jobs = [
   job(