You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by wf...@apache.org on 2015/09/11 19:11:38 UTC

aurora git commit: Revert "Revert "Refer to shared task_configs table for job updates.""

Repository: aurora
Updated Branches:
  refs/heads/master 0fc7c5a0e -> 12300292d


Revert "Revert "Refer to shared task_configs table for job updates.""

Bugs closed: AURORA-647

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


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

Branch: refs/heads/master
Commit: 12300292dcaa91a3cdebbb565fd61f1231b1df95
Parents: 0fc7c5a
Author: Bill Farner <wf...@apache.org>
Authored: Fri Sep 11 10:11:16 2015 -0700
Committer: Bill Farner <wf...@apache.org>
Committed: Fri Sep 11 10:11:28 2015 -0700

----------------------------------------------------------------------
 .../org/apache/aurora/benchmark/JobUpdates.java |   8 +-
 .../scheduler/storage/db/DbJobUpdateStore.java  |  62 +++----
 .../storage/db/JobUpdateDetailsMapper.java      |  22 +--
 .../db/typehandlers/TaskConfigTypeHandler.java  |  73 --------
 .../storage/db/typehandlers/TypeHandlers.java   |   1 -
 .../storage/db/views/DbInstanceTaskConfig.java  |  33 ++++
 .../scheduler/storage/db/views/DbJobUpdate.java |  36 ++++
 .../storage/db/views/DbJobUpdateDetails.java    |  33 ++++
 .../db/views/DbJobUpdateInstructions.java       |  45 +++++
 .../db/views/DbStoredJobUpdateDetails.java      |  30 +++
 .../storage/db/JobUpdateDetailsMapper.xml       |  35 ++--
 .../aurora/scheduler/storage/db/schema.sql      | 182 +++++++++----------
 .../storage/db/DbJobUpdateStoreTest.java        |   4 +-
 13 files changed, 332 insertions(+), 232 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/12300292/src/jmh/java/org/apache/aurora/benchmark/JobUpdates.java
----------------------------------------------------------------------
diff --git a/src/jmh/java/org/apache/aurora/benchmark/JobUpdates.java b/src/jmh/java/org/apache/aurora/benchmark/JobUpdates.java
index 48bee50..493f75d 100644
--- a/src/jmh/java/org/apache/aurora/benchmark/JobUpdates.java
+++ b/src/jmh/java/org/apache/aurora/benchmark/JobUpdates.java
@@ -20,7 +20,6 @@ import java.util.UUID;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 
-import org.apache.aurora.gen.ExecutorConfig;
 import org.apache.aurora.gen.InstanceTaskConfig;
 import org.apache.aurora.gen.JobInstanceUpdateEvent;
 import org.apache.aurora.gen.JobKey;
@@ -35,6 +34,8 @@ import org.apache.aurora.gen.JobUpdateStatus;
 import org.apache.aurora.gen.JobUpdateSummary;
 import org.apache.aurora.gen.Range;
 import org.apache.aurora.gen.TaskConfig;
+import org.apache.aurora.scheduler.base.TaskTestUtil;
+import org.apache.aurora.scheduler.storage.entities.IJobKey;
 import org.apache.aurora.scheduler.storage.entities.IJobUpdateDetails;
 
 /**
@@ -63,9 +64,8 @@ final class JobUpdates {
         JobKey job = new JobKey("role", "env", UUID.randomUUID().toString());
         JobUpdateKey key = new JobUpdateKey().setJob(job).setId(UUID.randomUUID().toString());
 
-        TaskConfig task = new TaskConfig()
-            .setJob(job)
-            .setExecutorConfig(new ExecutorConfig("cfg", string(10000)));
+        TaskConfig task = TaskTestUtil.makeConfig(IJobKey.build(job)).newBuilder();
+        task.getExecutorConfig().setData(string(10000));
 
         JobUpdate update = new JobUpdate()
             .setSummary(new JobUpdateSummary()

http://git-wip-us.apache.org/repos/asf/aurora/blob/12300292/src/main/java/org/apache/aurora/scheduler/storage/db/DbJobUpdateStore.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/DbJobUpdateStore.java b/src/main/java/org/apache/aurora/scheduler/storage/db/DbJobUpdateStore.java
index fd46c3f..7652132 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/db/DbJobUpdateStore.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/DbJobUpdateStore.java
@@ -25,12 +25,13 @@ import com.google.common.collect.FluentIterable;
 import com.google.common.collect.ImmutableSet;
 
 import org.apache.aurora.common.base.MorePreconditions;
-import org.apache.aurora.gen.JobUpdate;
-import org.apache.aurora.gen.JobUpdateInstructions;
 import org.apache.aurora.gen.JobUpdateStatus;
 import org.apache.aurora.gen.storage.StoredJobUpdateDetails;
 import org.apache.aurora.scheduler.stats.CachedCounters;
 import org.apache.aurora.scheduler.storage.JobUpdateStore;
+import org.apache.aurora.scheduler.storage.db.views.DbJobUpdate;
+import org.apache.aurora.scheduler.storage.db.views.DbJobUpdateInstructions;
+import org.apache.aurora.scheduler.storage.db.views.DbStoredJobUpdateDetails;
 import org.apache.aurora.scheduler.storage.entities.IInstanceTaskConfig;
 import org.apache.aurora.scheduler.storage.entities.IJobInstanceUpdateEvent;
 import org.apache.aurora.scheduler.storage.entities.IJobUpdate;
@@ -55,6 +56,7 @@ public class DbJobUpdateStore implements JobUpdateStore.Mutable {
   private final JobUpdateDetailsMapper detailsMapper;
   private final JobUpdateEventMapper jobEventMapper;
   private final JobInstanceUpdateEventMapper instanceEventMapper;
+  private final TaskConfigManager taskConfigManager;
   private final CachedCounters stats;
 
   @Inject
@@ -63,12 +65,14 @@ public class DbJobUpdateStore implements JobUpdateStore.Mutable {
       JobUpdateDetailsMapper detailsMapper,
       JobUpdateEventMapper jobEventMapper,
       JobInstanceUpdateEventMapper instanceEventMapper,
+      TaskConfigManager taskConfigManager,
       CachedCounters stats) {
 
     this.jobKeyMapper = requireNonNull(jobKeyMapper);
     this.detailsMapper = requireNonNull(detailsMapper);
     this.jobEventMapper = requireNonNull(jobEventMapper);
     this.instanceEventMapper = requireNonNull(instanceEventMapper);
+    this.taskConfigManager = requireNonNull(taskConfigManager);
     this.stats = requireNonNull(stats);
   }
 
@@ -103,7 +107,7 @@ public class DbJobUpdateStore implements JobUpdateStore.Mutable {
       IInstanceTaskConfig desired = update.getInstructions().getDesiredState();
       detailsMapper.insertTaskConfig(
           key,
-          desired.getTask().newBuilder(),
+          taskConfigManager.insert(desired.getTask()),
           true,
           new InsertResult());
 
@@ -116,7 +120,11 @@ public class DbJobUpdateStore implements JobUpdateStore.Mutable {
     if (!update.getInstructions().getInitialState().isEmpty()) {
       for (IInstanceTaskConfig config : update.getInstructions().getInitialState()) {
         InsertResult result = new InsertResult();
-        detailsMapper.insertTaskConfig(key, config.getTask().newBuilder(), false, result);
+        detailsMapper.insertTaskConfig(
+            key,
+            taskConfigManager.insert(config.getTask()),
+            false,
+            result);
 
         detailsMapper.insertTaskConfigInstances(
             result.getId(),
@@ -149,13 +157,6 @@ public class DbJobUpdateStore implements JobUpdateStore.Mutable {
     detailsMapper.truncate();
   }
 
-  private static final Function<PruneVictim, Long> GET_ROW_ID = new Function<PruneVictim, Long>() {
-    @Override
-    public Long apply(PruneVictim victim) {
-      return victim.getRowId();
-    }
-  };
-
   private static final Function<PruneVictim, IJobUpdateKey> GET_UPDATE_KEY =
       new Function<PruneVictim, IJobUpdateKey>() {
         @Override
@@ -180,7 +181,7 @@ public class DbJobUpdateStore implements JobUpdateStore.Mutable {
           historyPruneThresholdMs);
 
       detailsMapper.deleteCompletedUpdates(
-          FluentIterable.from(pruneVictims).transform(GET_ROW_ID).toSet());
+          FluentIterable.from(pruneVictims).transform(PruneVictim::getRowId).toSet());
       pruned.addAll(FluentIterable.from(pruneVictims).transform(GET_UPDATE_KEY));
     }
 
@@ -198,54 +199,41 @@ public class DbJobUpdateStore implements JobUpdateStore.Mutable {
   public List<IJobUpdateDetails> fetchJobUpdateDetails(IJobUpdateQuery query) {
     return FluentIterable
         .from(detailsMapper.selectDetailsList(query.newBuilder()))
-        .transform(new Function<StoredJobUpdateDetails, IJobUpdateDetails>() {
-          @Override
-          public IJobUpdateDetails apply(StoredJobUpdateDetails input) {
-            return IJobUpdateDetails.build(input.getDetails());
-          }
-        }).toList();
+        .transform(DbStoredJobUpdateDetails::toThrift)
+        .transform(StoredJobUpdateDetails::getDetails)
+        .transform(IJobUpdateDetails::build)
+        .toList();
   }
 
   @Timed("job_update_store_fetch_details")
   @Override
   public Optional<IJobUpdateDetails> fetchJobUpdateDetails(final IJobUpdateKey key) {
     return Optional.fromNullable(detailsMapper.selectDetails(key))
-        .transform(new Function<StoredJobUpdateDetails, IJobUpdateDetails>() {
-          @Override
-          public IJobUpdateDetails apply(StoredJobUpdateDetails input) {
-            return IJobUpdateDetails.build(input.getDetails());
-          }
-        });
+        .transform(DbStoredJobUpdateDetails::toThrift)
+        .transform(StoredJobUpdateDetails::getDetails)
+        .transform(IJobUpdateDetails::build);
   }
 
   @Timed("job_update_store_fetch_update")
   @Override
   public Optional<IJobUpdate> fetchJobUpdate(IJobUpdateKey key) {
     return Optional.fromNullable(detailsMapper.selectUpdate(key))
-        .transform(new Function<JobUpdate, IJobUpdate>() {
-          @Override
-          public IJobUpdate apply(JobUpdate input) {
-            return IJobUpdate.build(input);
-          }
-        });
+        .transform(DbJobUpdate::toImmutable);
   }
 
   @Timed("job_update_store_fetch_instructions")
   @Override
   public Optional<IJobUpdateInstructions> fetchJobUpdateInstructions(IJobUpdateKey key) {
     return Optional.fromNullable(detailsMapper.selectInstructions(key))
-        .transform(new Function<JobUpdateInstructions, IJobUpdateInstructions>() {
-          @Override
-          public IJobUpdateInstructions apply(JobUpdateInstructions input) {
-            return IJobUpdateInstructions.build(input);
-          }
-        });
+        .transform(DbJobUpdateInstructions::toImmutable);
   }
 
   @Timed("job_update_store_fetch_all_details")
   @Override
   public Set<StoredJobUpdateDetails> fetchAllJobUpdateDetails() {
-    return ImmutableSet.copyOf(detailsMapper.selectAllDetails());
+    return FluentIterable.from(detailsMapper.selectAllDetails())
+        .transform(DbStoredJobUpdateDetails::toThrift)
+        .toSet();
   }
 
   @Timed("job_update_store_get_lock_token")

http://git-wip-us.apache.org/repos/asf/aurora/blob/12300292/src/main/java/org/apache/aurora/scheduler/storage/db/JobUpdateDetailsMapper.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/JobUpdateDetailsMapper.java b/src/main/java/org/apache/aurora/scheduler/storage/db/JobUpdateDetailsMapper.java
index 02ea355..a3b0494 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/db/JobUpdateDetailsMapper.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/JobUpdateDetailsMapper.java
@@ -20,12 +20,12 @@ import javax.annotation.Nullable;
 
 import org.apache.aurora.gen.JobInstanceUpdateEvent;
 import org.apache.aurora.gen.JobUpdate;
-import org.apache.aurora.gen.JobUpdateInstructions;
 import org.apache.aurora.gen.JobUpdateQuery;
 import org.apache.aurora.gen.JobUpdateSummary;
 import org.apache.aurora.gen.Range;
-import org.apache.aurora.gen.TaskConfig;
-import org.apache.aurora.gen.storage.StoredJobUpdateDetails;
+import org.apache.aurora.scheduler.storage.db.views.DbJobUpdate;
+import org.apache.aurora.scheduler.storage.db.views.DbJobUpdateInstructions;
+import org.apache.aurora.scheduler.storage.db.views.DbStoredJobUpdateDetails;
 import org.apache.aurora.scheduler.storage.entities.IJobUpdateKey;
 import org.apache.ibatis.annotations.Param;
 
@@ -56,21 +56,21 @@ interface JobUpdateDetailsMapper {
    * Inserts a task configuration entry for an update.
    *
    * @param key Update to insert task configs for.
-   * @param taskConfig task configuration to insert.
+   * @param taskConfigRow task configuration row.
    * @param isNew Flag to identify if the task config is existing {@code false} or
    *              desired {@code true}.
    * @param result Container for auto-generated ID of the inserted job update row.
    */
   void insertTaskConfig(
       @Param("key") IJobUpdateKey key,
-      @Param("config") TaskConfig taskConfig,
+      @Param("taskConfigRow") long taskConfigRow,
       @Param("isNew") boolean isNew,
       @Param("result") InsertResult result);
 
   /**
    * Maps inserted task config with a set of associated instance ranges.
    *
-   * @param configId ID of the {@link TaskConfig} stored.
+   * @param configId ID of the task config stored.
    * @param ranges Set of instance ID ranges.
    */
   void insertTaskConfigInstances(
@@ -150,7 +150,7 @@ interface JobUpdateDetailsMapper {
    * @return Job update details for the provided update ID, if it exists.
    */
   @Nullable
-  StoredJobUpdateDetails selectDetails(@Param("key") IJobUpdateKey key);
+  DbStoredJobUpdateDetails selectDetails(@Param("key") IJobUpdateKey key);
 
   /**
    * Gets all job update details matching the provided {@code query}.
@@ -159,7 +159,7 @@ interface JobUpdateDetailsMapper {
    * @param query Query to filter results by.
    * @return Job update details matching the query.
    */
-  List<StoredJobUpdateDetails> selectDetailsList(JobUpdateQuery query);
+  List<DbStoredJobUpdateDetails> selectDetailsList(JobUpdateQuery query);
 
   /**
    * Gets job update for the provided {@code update}.
@@ -168,7 +168,7 @@ interface JobUpdateDetailsMapper {
    * @return Job update for the provided update ID, if it exists.
    */
   @Nullable
-  JobUpdate selectUpdate(@Param("key") IJobUpdateKey key);
+  DbJobUpdate selectUpdate(@Param("key") IJobUpdateKey key);
 
   /**
    * Gets job update instructions for the provided {@code update}.
@@ -177,14 +177,14 @@ interface JobUpdateDetailsMapper {
    * @return Job update instructions for the provided update ID, if it exists.
    */
   @Nullable
-  JobUpdateInstructions selectInstructions(@Param("key") IJobUpdateKey key);
+  DbJobUpdateInstructions selectInstructions(@Param("key") IJobUpdateKey key);
 
   /**
    * Gets all stored job update details.
    *
    * @return All stored job update details.
    */
-  Set<StoredJobUpdateDetails> selectAllDetails();
+  Set<DbStoredJobUpdateDetails> selectAllDetails();
 
   /**
    * Gets the token associated with an update.

http://git-wip-us.apache.org/repos/asf/aurora/blob/12300292/src/main/java/org/apache/aurora/scheduler/storage/db/typehandlers/TaskConfigTypeHandler.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/typehandlers/TaskConfigTypeHandler.java b/src/main/java/org/apache/aurora/scheduler/storage/db/typehandlers/TaskConfigTypeHandler.java
deleted file mode 100644
index 2f58357..0000000
--- a/src/main/java/org/apache/aurora/scheduler/storage/db/typehandlers/TaskConfigTypeHandler.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/**
- * 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.typehandlers;
-
-import java.sql.CallableStatement;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-
-import org.apache.aurora.codec.ThriftBinaryCodec;
-import org.apache.aurora.codec.ThriftBinaryCodec.CodingException;
-import org.apache.aurora.gen.TaskConfig;
-import org.apache.ibatis.type.JdbcType;
-import org.apache.ibatis.type.TypeHandler;
-
-/**
- * Type handler for objects of type {@link TaskConfig}. Converts {@link TaskConfig} to/from byte
- * array to be stored in SQL as BINARY type.
- *
- * <p/>
- * NOTE: We don't want to store serialized thrift objects long-term, but instead plan to reference
- * a canonical table of task configurations. This class will go away with AURORA-647.
- */
-class TaskConfigTypeHandler implements TypeHandler<TaskConfig> {
-
-  @Override
-  public final void setParameter(
-      PreparedStatement ps,
-      int i,
-      TaskConfig parameter,
-      JdbcType jdbcType) throws SQLException {
-
-    try {
-      ps.setBytes(i, ThriftBinaryCodec.encodeNonNull(parameter));
-    } catch (CodingException e) {
-      throw new SQLException("Failed to encode thrift struct.", e);
-    }
-  }
-
-  @Override
-  public final TaskConfig getResult(ResultSet rs, String columnName) throws SQLException {
-    return decodeOrThrow(rs.getBytes(columnName));
-  }
-
-  @Override
-  public final TaskConfig getResult(ResultSet rs, int columnIndex) throws SQLException {
-    return decodeOrThrow(rs.getBytes(columnIndex));
-  }
-
-  @Override
-  public final TaskConfig getResult(CallableStatement cs, int columnIndex) throws SQLException {
-    return decodeOrThrow(cs.getBytes(columnIndex));
-  }
-
-  private TaskConfig decodeOrThrow(byte[] value) throws SQLException {
-    try {
-      return ThriftBinaryCodec.decode(TaskConfig.class, value);
-    } catch (CodingException e) {
-      throw new SQLException("Failed to decode thrift struct.", e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/aurora/blob/12300292/src/main/java/org/apache/aurora/scheduler/storage/db/typehandlers/TypeHandlers.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/typehandlers/TypeHandlers.java b/src/main/java/org/apache/aurora/scheduler/storage/db/typehandlers/TypeHandlers.java
index 9afc3f3..ed561c6 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/db/typehandlers/TypeHandlers.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/typehandlers/TypeHandlers.java
@@ -34,7 +34,6 @@ public final class TypeHandlers {
         .add(JobUpdateStatusTypeHandler.class)
         .add(MaintenanceModeTypeHandler.class)
         .add(ScheduleStatusTypeHandler.class)
-        .add(TaskConfigTypeHandler.class)
         .build();
   }
 }

http://git-wip-us.apache.org/repos/asf/aurora/blob/12300292/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbInstanceTaskConfig.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbInstanceTaskConfig.java b/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbInstanceTaskConfig.java
new file mode 100644
index 0000000..f3fd7a9
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbInstanceTaskConfig.java
@@ -0,0 +1,33 @@
+/**
+ * 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 java.util.Set;
+
+import org.apache.aurora.gen.InstanceTaskConfig;
+import org.apache.aurora.gen.Range;
+
+public final class DbInstanceTaskConfig {
+  private DbTaskConfig task;
+  private Set<Range> instances;
+
+  private DbInstanceTaskConfig() {
+  }
+
+  InstanceTaskConfig toThrift() {
+    return new InstanceTaskConfig()
+        .setTask(task.toThrift())
+        .setInstances(instances);
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/12300292/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbJobUpdate.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbJobUpdate.java b/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbJobUpdate.java
new file mode 100644
index 0000000..78703e9
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbJobUpdate.java
@@ -0,0 +1,36 @@
+/**
+ * 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.JobUpdate;
+import org.apache.aurora.gen.JobUpdateSummary;
+import org.apache.aurora.scheduler.storage.entities.IJobUpdate;
+
+public final class DbJobUpdate {
+  private JobUpdateSummary summary;
+  private DbJobUpdateInstructions instructions;
+
+  private DbJobUpdate() {
+  }
+
+  JobUpdate toThrift() {
+    return new JobUpdate()
+        .setSummary(summary)
+        .setInstructions(instructions.toThrift());
+  }
+
+  public IJobUpdate toImmutable() {
+    return IJobUpdate.build(toThrift());
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/12300292/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbJobUpdateDetails.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbJobUpdateDetails.java b/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbJobUpdateDetails.java
new file mode 100644
index 0000000..3a52724
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbJobUpdateDetails.java
@@ -0,0 +1,33 @@
+/**
+ * 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 java.util.List;
+
+import org.apache.aurora.gen.JobInstanceUpdateEvent;
+import org.apache.aurora.gen.JobUpdateDetails;
+import org.apache.aurora.gen.JobUpdateEvent;
+
+public final class DbJobUpdateDetails {
+  private DbJobUpdate update;
+  private List<JobUpdateEvent> updateEvents;
+  private List<JobInstanceUpdateEvent> instanceEvents;
+
+  public JobUpdateDetails toThrift() {
+    return new JobUpdateDetails()
+        .setUpdate(update.toThrift())
+        .setUpdateEvents(updateEvents)
+        .setInstanceEvents(instanceEvents);
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/12300292/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbJobUpdateInstructions.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbJobUpdateInstructions.java b/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbJobUpdateInstructions.java
new file mode 100644
index 0000000..d19aa85
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbJobUpdateInstructions.java
@@ -0,0 +1,45 @@
+/**
+ * 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 java.util.Set;
+
+import com.google.common.collect.FluentIterable;
+
+import org.apache.aurora.gen.JobUpdateInstructions;
+import org.apache.aurora.gen.JobUpdateSettings;
+import org.apache.aurora.scheduler.storage.entities.IJobUpdateInstructions;
+
+public final class DbJobUpdateInstructions {
+  private Set<DbInstanceTaskConfig> initialState;
+  private DbInstanceTaskConfig desiredState;
+  private JobUpdateSettings settings;
+
+  private DbJobUpdateInstructions() {
+  }
+
+  JobUpdateInstructions toThrift() {
+    return new JobUpdateInstructions()
+        .setInitialState(
+            FluentIterable.from(initialState)
+                .transform(DbInstanceTaskConfig::toThrift)
+                .toSet())
+        .setDesiredState(desiredState == null ? null : desiredState.toThrift())
+        .setSettings(settings);
+  }
+
+  public IJobUpdateInstructions toImmutable() {
+    return IJobUpdateInstructions.build(toThrift());
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/12300292/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbStoredJobUpdateDetails.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbStoredJobUpdateDetails.java b/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbStoredJobUpdateDetails.java
new file mode 100644
index 0000000..8ec6d47
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbStoredJobUpdateDetails.java
@@ -0,0 +1,30 @@
+/**
+ * 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.storage.StoredJobUpdateDetails;
+
+public final class DbStoredJobUpdateDetails {
+  private DbJobUpdateDetails details;
+  private String lockToken;
+
+  private DbStoredJobUpdateDetails() {
+  }
+
+  public StoredJobUpdateDetails toThrift() {
+    return new StoredJobUpdateDetails()
+        .setDetails(details.toThrift())
+        .setLockToken(lockToken);
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/12300292/src/main/resources/org/apache/aurora/scheduler/storage/db/JobUpdateDetailsMapper.xml
----------------------------------------------------------------------
diff --git a/src/main/resources/org/apache/aurora/scheduler/storage/db/JobUpdateDetailsMapper.xml b/src/main/resources/org/apache/aurora/scheduler/storage/db/JobUpdateDetailsMapper.xml
index 6ffb54f..69e11b6 100644
--- a/src/main/resources/org/apache/aurora/scheduler/storage/db/JobUpdateDetailsMapper.xml
+++ b/src/main/resources/org/apache/aurora/scheduler/storage/db/JobUpdateDetailsMapper.xml
@@ -88,11 +88,11 @@
   <insert id="insertTaskConfig" useGeneratedKeys="true" keyColumn="id" keyProperty="result.id">
     INSERT INTO job_update_configs (
       update_row_id,
-      task_config,
+      task_config_row_id,
       is_new
     ) VALUES (
       <include refid="select_update_row_id"/>,
-      #{config, typeHandler=org.apache.aurora.scheduler.storage.db.typehandlers.TaskConfigTypeHandler},
+      #{taskConfigRow},
       #{isNew}
     )
   </insert>
@@ -157,11 +157,15 @@
     <id column="id" />
   </resultMap>
 
-  <resultMap id="instanceConfigMap" type="org.apache.aurora.gen.InstanceTaskConfig">
+  <resultMap
+      id="instanceConfigMap"
+      type="org.apache.aurora.scheduler.storage.db.views.DbInstanceTaskConfig">
     <id column="id" />
-    <result property="task"
-            column="task"
-            typeHandler="org.apache.aurora.scheduler.storage.db.typehandlers.TaskConfigTypeHandler" />
+    <association
+        property="task"
+        select="org.apache.aurora.scheduler.storage.db.TaskConfigMapper.selectConfig"
+        column="task_config_row_id"
+        foreignColumn="row_id"/>
     <collection property="instances" resultMap="rangeMap" columnPrefix="r_" notNullColumn="id" />
   </resultMap>
 
@@ -173,7 +177,9 @@
                 notNullColumn="id" />
   </resultMap>
 
-  <resultMap id="jobUpdateInstructionMap" type="org.apache.aurora.gen.JobUpdateInstructions">
+  <resultMap
+      id="jobUpdateInstructionMap"
+      type="org.apache.aurora.scheduler.storage.db.views.DbJobUpdateInstructions">
     <id column="id" />
     <association property="desiredState" resultMap="instanceConfigMap" columnPrefix="ditc_" />
     <association property="settings" resultMap="jobUpdateSettingsMap" columnPrefix="juse_"/>
@@ -183,7 +189,7 @@
                 notNullColumn="id" />
   </resultMap>
 
-  <resultMap id="jobUpdateMap" type="org.apache.aurora.gen.JobUpdate">
+  <resultMap id="jobUpdateMap" type="org.apache.aurora.scheduler.storage.db.views.DbJobUpdate">
     <id column="u_id" />
     <association property="summary" resultMap="jobUpdateSummaryMap" columnPrefix="jusm_"/>
     <association property="instructions" resultMap="jobUpdateInstructionMap" columnPrefix="jui_"/>
@@ -203,8 +209,9 @@
             typeHandler="org.apache.aurora.scheduler.storage.db.typehandlers.JobUpdateStatusTypeHandler"/>
   </resultMap>
 
-
-  <resultMap id="jobUpdateDetailsMap" type="org.apache.aurora.gen.storage.StoredJobUpdateDetails">
+  <resultMap
+      id="jobUpdateDetailsMap"
+      type="org.apache.aurora.scheduler.storage.db.views.DbStoredJobUpdateDetails">
     <id column="u_id" />
     <association property="details.update" resultMap="jobUpdateMap" />
     <collection property="details.updateEvents"
@@ -343,12 +350,12 @@
       u.block_if_no_pulses_after_ms AS jui_juse_block_if_no_pulses_after_ms,
       u.id AS jui_id,
       cn.id AS jui_ditc_id,
-      cn.task_config AS jui_ditc_task,
+      cn.task_config_row_id AS jui_ditc_task_config_row_id,
       di.id AS jui_ditc_r_id,
       di.first AS jui_ditc_r_first,
       di.last AS jui_ditc_r_last,
       co.id AS jui_iitc_id,
-      co.task_config AS jui_iitc_task,
+      co.task_config_row_id AS jui_iitc_task_config_row_id,
       ci.id AS jui_iitc_r_id,
       ci.first AS jui_iitc_r_first,
       ci.last AS jui_iitc_r_last,
@@ -396,12 +403,12 @@
       u.block_if_no_pulses_after_ms AS juse_block_if_no_pulses_after_ms,
       u.id AS id,
       cn.id AS ditc_id,
-      cn.task_config AS ditc_task,
+      cn.task_config_row_id AS ditc_task_config_row_id,
       di.id AS ditc_r_id,
       di.first AS ditc_r_first,
       di.last AS ditc_r_last,
       co.id AS iitc_id,
-      co.task_config AS iitc_task,
+      co.task_config_row_id AS iitc_task_config_row_id,
       ci.id AS iitc_r_id,
       ci.first AS iitc_r_first,
       ci.last AS iitc_r_last,

http://git-wip-us.apache.org/repos/asf/aurora/blob/12300292/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 d971aa1..4df685a 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
@@ -79,97 +79,6 @@ CREATE TABLE host_attribute_values(
   UNIQUE(host_attribute_id, name, value)
 );
 
-CREATE TABLE job_instance_update_actions(
-  id INT PRIMARY KEY,
-  name VARCHAR NOT NULL,
-
-  UNIQUE(name)
-);
-
-CREATE TABLE job_update_statuses(
-  id INT PRIMARY KEY,
-  name VARCHAR NOT NULL,
-
-  UNIQUE(name)
-);
-
-CREATE TABLE job_updates(
-  id IDENTITY,
-  job_key_id BIGINT NOT NULL REFERENCES job_keys(id),
-  update_id VARCHAR NOT NULL,
-  user VARCHAR NOT NULL,
-  update_group_size INT NOT NULL,
-  max_per_instance_failures INT NOT NULL,
-  max_failed_instances INT NOT NULL,
-  max_wait_to_instance_running_ms INT NOT NULL,
-  min_wait_in_instance_running_ms INT NOT NULL,
-  rollback_on_failure BOOLEAN NOT NULL,
-  wait_for_batch_completion BOOLEAN NOT NULL,
-  block_if_no_pulses_after_ms INT NULL,
-
-  UNIQUE(update_id, job_key_id)
-);
-
-CREATE TABLE job_update_locks(
-  id IDENTITY,
-  update_row_id BIGINT NOT NULL REFERENCES job_updates(id) ON DELETE CASCADE,
-  lock_token VARCHAR NOT NULL REFERENCES locks(token) ON DELETE CASCADE,
-
-  UNIQUE(update_row_id),
-  UNIQUE(lock_token)
-);
-
-CREATE TABLE job_update_configs(
-  id IDENTITY,
-  update_row_id BIGINT NOT NULL REFERENCES job_updates(id) ON DELETE CASCADE,
-  task_config BINARY NOT NULL,
-  is_new BOOLEAN NOT NULL
-);
-
-CREATE TABLE job_updates_to_instance_overrides(
-  id IDENTITY,
-  update_row_id BIGINT NOT NULL REFERENCES job_updates(id) ON DELETE CASCADE,
-  first INT NOT NULL,
-  last INT NOT NULL,
-
-  UNIQUE(update_row_id, first, last)
-);
-
-CREATE TABLE job_updates_to_desired_instances(
-  id IDENTITY,
-  update_row_id BIGINT NOT NULL REFERENCES job_updates(id) ON DELETE CASCADE,
-  first INT NOT NULL,
-  last INT NOT NULL,
-
-  UNIQUE(update_row_id, first, last)
-);
-
-CREATE TABLE job_update_configs_to_instances(
-  id IDENTITY,
-  config_id BIGINT NOT NULL REFERENCES job_update_configs(id) ON DELETE CASCADE,
-  first INT NOT NULL,
-  last INT NOT NULL,
-
-  UNIQUE(config_id, first, last)
-);
-
-CREATE TABLE job_update_events(
-  id IDENTITY,
-  update_row_id BIGINT NOT NULL REFERENCES job_updates(id) ON DELETE CASCADE,
-  status INT NOT NULL REFERENCES job_update_statuses(id),
-  timestamp_ms BIGINT NOT NULL,
-  user VARCHAR,
-  message VARCHAR
-);
-
-CREATE TABLE job_instance_update_events(
-  id IDENTITY,
-  update_row_id BIGINT NOT NULL REFERENCES job_updates(id) ON DELETE CASCADE,
-  action INT NOT NULL REFERENCES job_instance_update_actions(id),
-  instance_id INT NOT NULL,
-  timestamp_ms BIGINT NOT NULL
-);
-
 /**
  * NOTE: This table is truncated by TaskMapper, which will cause a conflict when the table is shared
  * with the forthcoming jobs table.  See note in TaskMapper about this before migrating MemJobStore.
@@ -318,3 +227,94 @@ CREATE TABLE cron_jobs(
 
   UNIQUE(job_key_id)
 );
+
+CREATE TABLE job_instance_update_actions(
+  id INT PRIMARY KEY,
+  name VARCHAR NOT NULL,
+
+  UNIQUE(name)
+);
+
+CREATE TABLE job_update_statuses(
+  id INT PRIMARY KEY,
+  name VARCHAR NOT NULL,
+
+  UNIQUE(name)
+);
+
+CREATE TABLE job_updates(
+  id IDENTITY,
+  job_key_id BIGINT NOT NULL REFERENCES job_keys(id),
+  update_id VARCHAR NOT NULL,
+  user VARCHAR NOT NULL,
+  update_group_size INT NOT NULL,
+  max_per_instance_failures INT NOT NULL,
+  max_failed_instances INT NOT NULL,
+  max_wait_to_instance_running_ms INT NOT NULL,
+  min_wait_in_instance_running_ms INT NOT NULL,
+  rollback_on_failure BOOLEAN NOT NULL,
+  wait_for_batch_completion BOOLEAN NOT NULL,
+  block_if_no_pulses_after_ms INT NULL,
+
+  UNIQUE(update_id, job_key_id)
+);
+
+CREATE TABLE job_update_locks(
+  id IDENTITY,
+  update_row_id BIGINT NOT NULL REFERENCES job_updates(id) ON DELETE CASCADE,
+  lock_token VARCHAR NOT NULL REFERENCES locks(token) ON DELETE CASCADE,
+
+  UNIQUE(update_row_id),
+  UNIQUE(lock_token)
+);
+
+CREATE TABLE job_update_configs(
+  id IDENTITY,
+  update_row_id BIGINT NOT NULL REFERENCES job_updates(id) ON DELETE CASCADE,
+  task_config_row_id INT NOT NULL REFERENCES task_configs(id),
+  is_new BOOLEAN NOT NULL
+);
+
+CREATE TABLE job_updates_to_instance_overrides(
+  id IDENTITY,
+  update_row_id BIGINT NOT NULL REFERENCES job_updates(id) ON DELETE CASCADE,
+  first INT NOT NULL,
+  last INT NOT NULL,
+
+  UNIQUE(update_row_id, first, last)
+);
+
+CREATE TABLE job_updates_to_desired_instances(
+  id IDENTITY,
+  update_row_id BIGINT NOT NULL REFERENCES job_updates(id) ON DELETE CASCADE,
+  first INT NOT NULL,
+  last INT NOT NULL,
+
+  UNIQUE(update_row_id, first, last)
+);
+
+CREATE TABLE job_update_configs_to_instances(
+  id IDENTITY,
+  config_id BIGINT NOT NULL REFERENCES job_update_configs(id) ON DELETE CASCADE,
+  first INT NOT NULL,
+  last INT NOT NULL,
+
+  UNIQUE(config_id, first, last)
+);
+
+CREATE TABLE job_update_events(
+  id IDENTITY,
+  update_row_id BIGINT NOT NULL REFERENCES job_updates(id) ON DELETE CASCADE,
+  status INT NOT NULL REFERENCES job_update_statuses(id),
+  timestamp_ms BIGINT NOT NULL,
+  user VARCHAR,
+  message VARCHAR
+);
+
+CREATE TABLE job_instance_update_events(
+  id IDENTITY,
+  update_row_id BIGINT NOT NULL REFERENCES job_updates(id) ON DELETE CASCADE,
+  action INT NOT NULL REFERENCES job_instance_update_actions(id),
+  instance_id INT NOT NULL,
+  timestamp_ms BIGINT NOT NULL
+);

http://git-wip-us.apache.org/repos/asf/aurora/blob/12300292/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 4a09693..c159378 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
@@ -214,7 +214,9 @@ public class DbJobUpdateStoreTest {
   public void testSaveEmptyInitialStateRangesThrows() {
     JobUpdate builder = makeJobUpdate(makeKey("u1")).newBuilder();
     builder.getInstructions().getInitialState().add(
-        new InstanceTaskConfig(new TaskConfig(), ImmutableSet.of()));
+        new InstanceTaskConfig(
+            TaskTestUtil.makeConfig(TaskTestUtil.JOB).newBuilder(),
+            ImmutableSet.of()));
 
     saveUpdate(IJobUpdate.build(builder), Optional.of("lock"));
   }