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 2014/08/08 18:24:43 UTC

git commit: Implementing SQL mappers for saveUpdate and fetchDetails.

Repository: incubator-aurora
Updated Branches:
  refs/heads/master 5154f61e7 -> 63792c44e


Implementing SQL mappers for saveUpdate and fetchDetails.

Bugs closed: AURORA-612

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


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

Branch: refs/heads/master
Commit: 63792c44ee7232f7c31ed2a966ff393755675f50
Parents: 5154f61
Author: Maxim Khutornenko <ma...@apache.org>
Authored: Fri Aug 8 09:24:21 2014 -0700
Committer: Maxim Khutornenko <ma...@apache.org>
Committed: Fri Aug 8 09:24:21 2014 -0700

----------------------------------------------------------------------
 .../scheduler/storage/db/DBJobUpdateStore.java  |  51 ++-
 .../scheduler/storage/db/DbLockStore.java       |  16 +-
 .../aurora/scheduler/storage/db/DbModule.java   |   9 +-
 .../aurora/scheduler/storage/db/DbStorage.java  |  22 +-
 .../db/JobInstanceUpdateEventMapper.java        |  33 ++
 .../scheduler/storage/db/JobKeyMapper.java      |   4 +-
 .../storage/db/JobUpdateDetailsMapper.java      |  48 +++
 .../storage/db/JobUpdateEventMapper.java        |  33 ++
 .../scheduler/storage/db/LockKeyMapper.java     |   4 +-
 .../JobUpdateActionTypeHandler.java             |  26 ++
 .../JobUpdateStatusTypeHandler.java             |  26 ++
 .../storage/db/typehandlers/TypeHandlers.java   |   5 +-
 .../storage/log/SnapshotStoreImpl.java          |   4 +-
 .../storage/db/JobInstanceUpdateEventMapper.xml |  37 +++
 .../scheduler/storage/db/JobKeyMapper.xml       |  10 +-
 .../storage/db/JobUpdateDetailsMapper.xml       | 140 +++++++++
 .../storage/db/JobUpdateEventMapper.xml         |  35 +++
 .../aurora/scheduler/storage/db/schema.sql      |  44 ++-
 .../thrift/org/apache/aurora/gen/api.thrift     |  16 +-
 .../storage/db/DBJobUpdateStoreTest.java        | 308 +++++++++++++++++++
 .../scheduler/storage/log/LogStorageTest.java   |   2 +-
 .../storage/log/SnapshotStoreImplTest.java      |   3 +-
 .../org/apache/aurora/gen/api.thrift.md5        |   2 +-
 23 files changed, 798 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/63792c44/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 869590e..87f428b 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
@@ -13,9 +13,13 @@
  */
 package org.apache.aurora.scheduler.storage.db;
 
+import javax.inject.Inject;
+
+import com.google.common.base.Function;
 import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableSet;
 
+import org.apache.aurora.gen.JobUpdateDetails;
 import org.apache.aurora.gen.JobUpdateQuery;
 import org.apache.aurora.scheduler.storage.JobUpdateStore;
 import org.apache.aurora.scheduler.storage.entities.IJobInstanceUpdateEvent;
@@ -24,41 +28,68 @@ import org.apache.aurora.scheduler.storage.entities.IJobUpdateDetails;
 import org.apache.aurora.scheduler.storage.entities.IJobUpdateEvent;
 import org.apache.aurora.scheduler.storage.entities.IJobUpdateSummary;
 
+import static java.util.Objects.requireNonNull;
+
 /**
  * A relational database-backed job update store.
  */
 public class DBJobUpdateStore implements JobUpdateStore.Mutable {
 
+  private final JobKeyMapper jobKeyMapper;
+  private final JobUpdateDetailsMapper detailsMapper;
+  private final JobUpdateEventMapper jobEventMapper;
+  private final JobInstanceUpdateEventMapper instanceEventMapper;
+
+  @Inject
+  DBJobUpdateStore(
+      JobKeyMapper jobKeyMapper,
+      JobUpdateDetailsMapper detailsMapper,
+      JobUpdateEventMapper jobEventMapper,
+      JobInstanceUpdateEventMapper instanceEventMapper) {
+
+    this.jobKeyMapper = requireNonNull(jobKeyMapper);
+    this.detailsMapper = requireNonNull(detailsMapper);
+    this.jobEventMapper = requireNonNull(jobEventMapper);
+    this.instanceEventMapper = requireNonNull(instanceEventMapper);
+  }
+
   @Override
   public void saveJobUpdate(IJobUpdate update) {
-    // TODO(maxim): implement DB mapping logic.
+    jobKeyMapper.merge(update.getSummary().getJobKey().newBuilder());
+    detailsMapper.merge(update.newBuilder());
   }
 
   @Override
   public void saveJobUpdateEvent(IJobUpdateEvent event, String updateId) {
-    // TODO(maxim): implement DB mapping logic.
+    jobEventMapper.insert(updateId, event.newBuilder());
   }
 
   @Override
   public void saveJobInstanceUpdateEvent(IJobInstanceUpdateEvent event, String updateId) {
-    // TODO(maxim): implement DB mapping logic.
+    instanceEventMapper.insert(event.newBuilder(), updateId);
   }
 
   @Override
-  public ImmutableSet<IJobUpdateSummary> fetchJobUpdateSummaries(JobUpdateQuery query) {
-    // TODO(maxim): implement DB mapping logic.
-    return ImmutableSet.of();
+  public void deleteAllUpdatesAndEvents() {
+    detailsMapper.truncate();
   }
 
   @Override
-  public void deleteAllUpdatesAndEvents() {
+  public ImmutableSet<IJobUpdateSummary> fetchJobUpdateSummaries(JobUpdateQuery query) {
     // TODO(maxim): implement DB mapping logic.
+    return ImmutableSet.of();
   }
 
   @Override
-  public Optional<IJobUpdateDetails> fetchJobUpdateDetails(String updateId) {
-    // TODO(maxim): implement DB mapping logic.
-    return Optional.absent();
+  public Optional<IJobUpdateDetails> fetchJobUpdateDetails(final String updateId) {
+    // TODO(maxim): add support for job_update_configs and update_only_these_instances.
+    return Optional.fromNullable(detailsMapper.selectDetails(updateId))
+        .transform(new Function<JobUpdateDetails, IJobUpdateDetails>() {
+          @Override
+          public IJobUpdateDetails apply(JobUpdateDetails input) {
+            return IJobUpdateDetails.build(input);
+          }
+        });
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/63792c44/src/main/java/org/apache/aurora/scheduler/storage/db/DbLockStore.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/DbLockStore.java b/src/main/java/org/apache/aurora/scheduler/storage/db/DbLockStore.java
index 5d4da0a..4b4f127 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/db/DbLockStore.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/DbLockStore.java
@@ -14,7 +14,6 @@
 package org.apache.aurora.scheduler.storage.db;
 
 import java.util.Set;
-import java.util.logging.Logger;
 
 import com.google.common.base.Function;
 import com.google.common.base.Optional;
@@ -25,7 +24,6 @@ import org.apache.aurora.scheduler.storage.LockStore;
 import org.apache.aurora.scheduler.storage.db.views.LockRow;
 import org.apache.aurora.scheduler.storage.entities.ILock;
 import org.apache.aurora.scheduler.storage.entities.ILockKey;
-import org.apache.ibatis.exceptions.PersistenceException;
 
 import static java.util.Objects.requireNonNull;
 
@@ -34,8 +32,6 @@ import static java.util.Objects.requireNonNull;
  */
 class DbLockStore implements LockStore.Mutable {
 
-  private static final Logger LOG = Logger.getLogger(DbLockStore.class.getName());
-
   private final LockMapper mapper;
   private final LockKeyMapper lockKeyMapper;
 
@@ -47,17 +43,7 @@ class DbLockStore implements LockStore.Mutable {
 
   @Override
   public void saveLock(ILock lock) {
-    try {
-      lockKeyMapper.insert(lock.getKey().newBuilder());
-    } catch (PersistenceException e) {
-      LOG.fine("DB write error for key: " + lock.getKey());
-      // TODO(davmclau): We purposely swallow duplicate key exceptions here
-      // but we should verify it _is_ a duplicate key error so we can
-      // give better logging for unexpected errors. That is
-      // made tricky by this: https://code.google.com/p/mybatis/issues/detail?id=249
-      // It is currently harmless to let this fall through, as if the
-      // write failed and key doesn't exist, the next write will fail anyway.
-    }
+    lockKeyMapper.insert(lock.getKey().newBuilder());
     mapper.insert(lock.newBuilder());
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/63792c44/src/main/java/org/apache/aurora/scheduler/storage/db/DbModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/DbModule.java b/src/main/java/org/apache/aurora/scheduler/storage/db/DbModule.java
index 5d65218..2a75646 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/db/DbModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/DbModule.java
@@ -105,14 +105,17 @@ public class DbModule extends PrivateModule {
         bindConstant().annotatedWith(named("JDBC.driver")).to(Driver.class.getName());
         bind(Key.get(String.class, named("JDBC.url"))).toInstance(jdbcUrl);
 
-        bindDataSourceProviderType(PooledDataSourceProvider.class);
         bindTransactionFactoryType(JdbcTransactionFactory.class);
+        bindDataSourceProviderType(PooledDataSourceProvider.class);
         addMapperClass(AttributeMapper.class);
         addMapperClass(EnumValueMapper.class);
-        addMapperClass(LockMapper.class);
+        addMapperClass(FrameworkIdMapper.class);
+        addMapperClass(JobInstanceUpdateEventMapper.class);
         addMapperClass(JobKeyMapper.class);
+        addMapperClass(JobUpdateEventMapper.class);
+        addMapperClass(JobUpdateDetailsMapper.class);
+        addMapperClass(LockMapper.class);
         addMapperClass(QuotaMapper.class);
-        addMapperClass(FrameworkIdMapper.class);
         Properties props = new Properties();
         // We have no plans to take advantage of multiple DB environments. This is a required
         // property though, so we use an unnamed environment.

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/63792c44/src/main/java/org/apache/aurora/scheduler/storage/db/DbStorage.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/DbStorage.java b/src/main/java/org/apache/aurora/scheduler/storage/db/DbStorage.java
index 8a573f7..71bc4c9 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/db/DbStorage.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/DbStorage.java
@@ -21,6 +21,8 @@ import com.google.common.io.CharStreams;
 import com.google.common.util.concurrent.AbstractIdleService;
 import com.google.inject.Inject;
 
+import org.apache.aurora.gen.JobUpdateAction;
+import org.apache.aurora.gen.JobUpdateStatus;
 import org.apache.aurora.gen.MaintenanceMode;
 import org.apache.aurora.scheduler.storage.AttributeStore;
 import org.apache.aurora.scheduler.storage.JobStore;
@@ -32,7 +34,7 @@ import org.apache.aurora.scheduler.storage.Storage;
 import org.apache.aurora.scheduler.storage.TaskStore;
 import org.apache.ibatis.builder.StaticSqlSource;
 import org.apache.ibatis.exceptions.PersistenceException;
-import org.apache.ibatis.logging.jdk14.Jdk14LoggingImpl;
+import org.apache.ibatis.logging.LogFactory;
 import org.apache.ibatis.mapping.MappedStatement.Builder;
 import org.apache.ibatis.mapping.SqlCommandType;
 import org.apache.ibatis.session.Configuration;
@@ -66,7 +68,8 @@ class DbStorage extends AbstractIdleService implements Storage {
       final SchedulerStore.Mutable schedulerStore,
       final AttributeStore.Mutable attributeStore,
       final LockStore.Mutable lockStore,
-      final QuotaStore.Mutable quotaStore) {
+      final QuotaStore.Mutable quotaStore,
+      final JobUpdateStore.Mutable jobUpdateStore) {
 
     this.sessionFactory = requireNonNull(sessionFactory);
     this.enumValueMapper = requireNonNull(enumValueMapper);
@@ -74,6 +77,7 @@ class DbStorage extends AbstractIdleService implements Storage {
     requireNonNull(attributeStore);
     requireNonNull(lockStore);
     requireNonNull(quotaStore);
+    requireNonNull(jobUpdateStore);
     storeProvider = new MutableStoreProvider() {
       @Override
       public SchedulerStore.Mutable getSchedulerStore() {
@@ -112,7 +116,7 @@ class DbStorage extends AbstractIdleService implements Storage {
 
       @Override
       public JobUpdateStore.Mutable getJobUpdateStore() {
-        throw new UnsupportedOperationException("Not implemented.");
+        return jobUpdateStore;
       }
     };
   }
@@ -161,6 +165,8 @@ class DbStorage extends AbstractIdleService implements Storage {
   @Override
   @Transactional
   protected void startUp() throws IOException {
+    LogFactory.useJdkLogging();
+
     Configuration configuration = sessionFactory.getConfiguration();
     String createStatementName = "create_tables";
     configuration.setMapUnderscoreToCamelCase(true);
@@ -176,8 +182,6 @@ class DbStorage extends AbstractIdleService implements Storage {
         SqlCommandType.UPDATE)
         .build());
 
-    configuration.setLogImpl(Jdk14LoggingImpl.class);
-
     try (SqlSession session = sessionFactory.openSession()) {
       session.update(createStatementName);
     }
@@ -185,6 +189,14 @@ class DbStorage extends AbstractIdleService implements Storage {
     for (MaintenanceMode mode : MaintenanceMode.values()) {
       enumValueMapper.addEnumValue("maintenance_modes", mode.getValue(), mode.name());
     }
+
+    for (JobUpdateStatus status : JobUpdateStatus.values()) {
+      enumValueMapper.addEnumValue("job_update_statuses", status.getValue(), status.name());
+    }
+
+    for (JobUpdateAction action : JobUpdateAction.values()) {
+      enumValueMapper.addEnumValue("job_instance_update_actions", action.getValue(), action.name());
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/63792c44/src/main/java/org/apache/aurora/scheduler/storage/db/JobInstanceUpdateEventMapper.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/JobInstanceUpdateEventMapper.java b/src/main/java/org/apache/aurora/scheduler/storage/db/JobInstanceUpdateEventMapper.java
new file mode 100644
index 0000000..d5dd5a5
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/JobInstanceUpdateEventMapper.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;
+
+import org.apache.aurora.gen.JobInstanceUpdateEvent;
+import org.apache.ibatis.annotations.Param;
+
+/**
+ * MyBatis mapper class for JobInstanceUpdateEventMapper.xml
+ *
+ * See http://mybatis.github.io/mybatis-3/sqlmap-xml.html for more details.
+ */
+interface JobInstanceUpdateEventMapper {
+
+  /**
+   * Inserts a new job instance update event into the database.
+   *
+   * @param event Event to insert.
+   * @param updateId Update ID of the event.
+   */
+  void insert(@Param("event") JobInstanceUpdateEvent event, @Param("updateId") String updateId);
+}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/63792c44/src/main/java/org/apache/aurora/scheduler/storage/db/JobKeyMapper.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/JobKeyMapper.java b/src/main/java/org/apache/aurora/scheduler/storage/db/JobKeyMapper.java
index 844714c..afdaa49 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/db/JobKeyMapper.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/JobKeyMapper.java
@@ -24,9 +24,9 @@ import org.apache.aurora.gen.JobKey;
  */
 interface JobKeyMapper {
   /**
-   * Inserts a new job key into the database.
+   * Saves the job key, updating the existing value if it exists.
    */
-  void insert(JobKey key);
+  void merge(JobKey key);
 
   /**
    * Permanently deletes a job key from the database.

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/63792c44/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
new file mode 100644
index 0000000..5b7a3df
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/JobUpdateDetailsMapper.java
@@ -0,0 +1,48 @@
+/**
+ * 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;
+
+import javax.annotation.Nullable;
+
+import org.apache.aurora.gen.JobUpdate;
+import org.apache.aurora.gen.JobUpdateDetails;
+
+/**
+ * MyBatis mapper class for JobUpdateDetailsMapper.xml
+ *
+ * See http://mybatis.github.io/mybatis-3/sqlmap-xml.html for more details.
+ */
+interface JobUpdateDetailsMapper {
+
+  /**
+   * Saves the job update, modifies the existing value if one exists.
+   *
+   * @param jobUpdate Job update to save/modify.
+   */
+  void merge(JobUpdate jobUpdate);
+
+  /**
+   * Deletes all updates and events from the database.
+   */
+  void truncate();
+
+  /**
+   * Gets {@link JobUpdateDetails} for the provided {@code updateId}.
+   *
+   * @param updateId Update ID to get.
+   * @return {@link JobUpdateDetails} instance, if it exists.
+   */
+  @Nullable
+  JobUpdateDetails selectDetails(String updateId);
+}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/63792c44/src/main/java/org/apache/aurora/scheduler/storage/db/JobUpdateEventMapper.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/JobUpdateEventMapper.java b/src/main/java/org/apache/aurora/scheduler/storage/db/JobUpdateEventMapper.java
new file mode 100644
index 0000000..bbd2f46
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/JobUpdateEventMapper.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;
+
+import org.apache.aurora.gen.JobUpdateEvent;
+import org.apache.ibatis.annotations.Param;
+
+/**
+ * MyBatis mapper class for JobUpdateEventMapper.xml
+ *
+ * See http://mybatis.github.io/mybatis-3/sqlmap-xml.html for more details.
+ */
+interface JobUpdateEventMapper {
+
+  /**
+   * Inserts a new job update event into the database.
+   *
+   * @param updateId Update ID of the event.
+   * @param event Event to insert.
+   */
+  void insert(@Param("updateId") String updateId, @Param("event") JobUpdateEvent event);
+}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/63792c44/src/main/java/org/apache/aurora/scheduler/storage/db/LockKeyMapper.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/LockKeyMapper.java b/src/main/java/org/apache/aurora/scheduler/storage/db/LockKeyMapper.java
index 618d5b7..5e9ba82 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/db/LockKeyMapper.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/LockKeyMapper.java
@@ -30,7 +30,7 @@ import static java.util.Objects.requireNonNull;
  * with the SQL to satisfy the interface.
  *
  */
-public class LockKeyMapper {
+class LockKeyMapper {
 
   private final JobKeyMapper jobKeyMapper;
 
@@ -41,7 +41,7 @@ public class LockKeyMapper {
 
   public void insert(LockKey key) {
     if (key.isSetJob()) {
-      jobKeyMapper.insert(requireNonNull(key.getJob()));
+      jobKeyMapper.merge(requireNonNull(key.getJob()));
     } else {
       throw new IllegalArgumentException("Unsupported lock type on LockKey.");
     }

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/63792c44/src/main/java/org/apache/aurora/scheduler/storage/db/typehandlers/JobUpdateActionTypeHandler.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/typehandlers/JobUpdateActionTypeHandler.java b/src/main/java/org/apache/aurora/scheduler/storage/db/typehandlers/JobUpdateActionTypeHandler.java
new file mode 100644
index 0000000..1b70684
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/typehandlers/JobUpdateActionTypeHandler.java
@@ -0,0 +1,26 @@
+/**
+ * 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 org.apache.aurora.gen.JobUpdateAction;
+
+/**
+ * Type handler for {@link JobUpdateAction}.
+ */
+class JobUpdateActionTypeHandler extends AbstractTEnumTypeHandler<JobUpdateAction> {
+  @Override
+  protected JobUpdateAction fromValue(int value) {
+    return JobUpdateAction.findByValue(value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/63792c44/src/main/java/org/apache/aurora/scheduler/storage/db/typehandlers/JobUpdateStatusTypeHandler.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/typehandlers/JobUpdateStatusTypeHandler.java b/src/main/java/org/apache/aurora/scheduler/storage/db/typehandlers/JobUpdateStatusTypeHandler.java
new file mode 100644
index 0000000..72f8b85
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/typehandlers/JobUpdateStatusTypeHandler.java
@@ -0,0 +1,26 @@
+/**
+ * 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 org.apache.aurora.gen.JobUpdateStatus;
+
+/**
+ * Type handler for {@link JobUpdateStatus}.
+ */
+class JobUpdateStatusTypeHandler extends AbstractTEnumTypeHandler<JobUpdateStatus> {
+  @Override
+  protected JobUpdateStatus fromValue(int value) {
+    return JobUpdateStatus.findByValue(value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/63792c44/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 f014123..c5468b1 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
@@ -28,6 +28,9 @@ public final class TypeHandlers {
   }
 
   public static List<Class<? extends TypeHandler<?>>> getAll() {
-    return ImmutableList.<Class<? extends TypeHandler<?>>>of(MaintenanceModeTypeHandler.class);
+    return ImmutableList.<Class<? extends TypeHandler<?>>>of(
+        JobUpdateActionTypeHandler.class,
+        JobUpdateStatusTypeHandler.class,
+        MaintenanceModeTypeHandler.class);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/63792c44/src/main/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl.java b/src/main/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl.java
index e921e71..3d291dd 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl.java
@@ -222,12 +222,12 @@ public class SnapshotStoreImpl implements SnapshotStore<Snapshot> {
               for (JobUpdateEvent updateEvent : details.getUpdateEvents()) {
                 updateStore.saveJobUpdateEvent(
                     IJobUpdateEvent.build(updateEvent),
-                    details.getUpdate().getUpdateId());
+                    details.getUpdate().getSummary().getUpdateId());
               }
               for (JobInstanceUpdateEvent instanceEvent : details.getInstanceEvents()) {
                 updateStore.saveJobInstanceUpdateEvent(
                     IJobInstanceUpdateEvent.build(instanceEvent),
-                    details.getUpdate().getUpdateId());
+                    details.getUpdate().getSummary().getUpdateId());
               }
             }
           }

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/63792c44/src/main/resources/org/apache/aurora/scheduler/storage/db/JobInstanceUpdateEventMapper.xml
----------------------------------------------------------------------
diff --git a/src/main/resources/org/apache/aurora/scheduler/storage/db/JobInstanceUpdateEventMapper.xml b/src/main/resources/org/apache/aurora/scheduler/storage/db/JobInstanceUpdateEventMapper.xml
new file mode 100644
index 0000000..1bc2a62
--- /dev/null
+++ b/src/main/resources/org/apache/aurora/scheduler/storage/db/JobInstanceUpdateEventMapper.xml
@@ -0,0 +1,37 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ 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.
+ -->
+
+<!DOCTYPE mapper
+        PUBLIC "-//mybatis.org//DTD Mapper 3.0//EN"
+        "http://mybatis.org/dtd/mybatis-3-mapper.dtd">
+<mapper namespace="org.apache.aurora.scheduler.storage.db.JobInstanceUpdateEventMapper">
+  <insert id="insert">
+    INSERT INTO job_instance_update_events (
+      update_id,
+      action,
+      instance_id,
+      timestamp_ms
+    ) VALUES (
+      (
+        SELECT ID
+        FROM job_updates
+        WHERE update_id = #{updateId}
+      ),
+      #{event.action, typeHandler=org.apache.aurora.scheduler.storage.db.typehandlers.JobUpdateActionTypeHandler},
+      #{event.instanceId},
+      #{event.timestampMs}
+    )
+  </insert>
+</mapper>

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/63792c44/src/main/resources/org/apache/aurora/scheduler/storage/db/JobKeyMapper.xml
----------------------------------------------------------------------
diff --git a/src/main/resources/org/apache/aurora/scheduler/storage/db/JobKeyMapper.xml b/src/main/resources/org/apache/aurora/scheduler/storage/db/JobKeyMapper.xml
index 3997f98..f5829ac 100644
--- a/src/main/resources/org/apache/aurora/scheduler/storage/db/JobKeyMapper.xml
+++ b/src/main/resources/org/apache/aurora/scheduler/storage/db/JobKeyMapper.xml
@@ -17,12 +17,12 @@
     PUBLIC "-//mybatis.org//DTD Mapper 3.0//EN"
     "http://mybatis.org/dtd/mybatis-3-mapper.dtd">
 <mapper namespace="org.apache.aurora.scheduler.storage.db.JobKeyMapper">
-  <insert id="insert">
-    INSERT INTO job_keys (
+  <insert id="merge">
+    MERGE INTO job_keys (
       role,
       environment,
       name
-    ) VALUES (
+    ) KEY(role, environment, name) VALUES (
       #{role},
       #{environment},
       #{name}
@@ -44,4 +44,8 @@
   <select id="selectAll" resultType="org.apache.aurora.gen.JobKey">
     SELECT * FROM job_keys
   </select>
+
+  <resultMap id="jobKeyMap" type="org.apache.aurora.gen.JobKey">
+    <id column="id" />
+  </resultMap>
 </mapper>

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/63792c44/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
new file mode 100644
index 0000000..3c69bda
--- /dev/null
+++ b/src/main/resources/org/apache/aurora/scheduler/storage/db/JobUpdateDetailsMapper.xml
@@ -0,0 +1,140 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ 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.
+ -->
+
+<!DOCTYPE mapper
+        PUBLIC "-//mybatis.org//DTD Mapper 3.0//EN"
+        "http://mybatis.org/dtd/mybatis-3-mapper.dtd">
+<mapper namespace="org.apache.aurora.scheduler.storage.db.JobUpdateDetailsMapper">
+  <insert id="merge">
+    MERGE INTO job_updates (
+      job_key_id,
+      update_id,
+      user,
+      status,
+      created_timestamp_ms,
+      last_modified_timestamp_ms,
+      update_group_size,
+      max_per_instance_failures,
+      max_failed_instances,
+      max_wait_to_instance_running_ms,
+      min_wait_in_instance_running_ms,
+      rollback_on_failure
+    ) KEY(update_id) VALUES (
+      (
+        SELECT ID
+        FROM job_keys
+        WHERE role = #{summary.jobKey.role}
+          AND environment = #{summary.jobKey.environment}
+          AND name = #{summary.jobKey.name}
+      ),
+      #{summary.updateId},
+      #{summary.user},
+      #{summary.status, typeHandler=org.apache.aurora.scheduler.storage.db.typehandlers.JobUpdateStatusTypeHandler},
+      #{summary.createdTimestampMs},
+      #{summary.lastModifiedTimestampMs},
+      #{configuration.settings.updateGroupSize},
+      #{configuration.settings.maxPerInstanceFailures},
+      #{configuration.settings.maxFailedInstances},
+      #{configuration.settings.maxWaitToInstanceRunningMs},
+      #{configuration.settings.minWaitInInstanceRunningMs},
+      #{configuration.settings.rollbackOnFailure}
+    )
+  </insert>
+
+  <resultMap id="jobUpdateSummaryMap" type="org.apache.aurora.gen.JobUpdateSummary">
+    <id column="update_id" property="updateId"/>
+    <result property="status"
+            column="status"
+            typeHandler="org.apache.aurora.scheduler.storage.db.typehandlers.JobUpdateStatusTypeHandler" />
+    <association property="jobKey"
+                 resultMap="org.apache.aurora.scheduler.storage.db.JobKeyMapper.jobKeyMap"
+                 columnPrefix="jk_"/>
+  </resultMap>
+
+  <resultMap id="jobUpdateSettingsMap" type="org.apache.aurora.gen.JobUpdateSettings">
+    <id column="created_timestamp_ms" />
+  </resultMap>
+
+  <resultMap id="jobUpdateConfigurationMap" type="org.apache.aurora.gen.JobUpdateConfiguration">
+    <association property="settings" resultMap="jobUpdateSettingsMap" columnPrefix="juse_"/>
+  </resultMap>
+
+  <resultMap id="jobUpdateMap" type="org.apache.aurora.gen.JobUpdate">
+    <association property="summary" resultMap="jobUpdateSummaryMap" columnPrefix="jusm_"/>
+    <association property="configuration" resultMap="jobUpdateConfigurationMap" />
+  </resultMap>
+
+  <resultMap id="jobUpdateDetailsMap" type="org.apache.aurora.gen.JobUpdateDetails">
+    <id column="u_id" />
+    <association property="update" resultMap="jobUpdateMap" />
+    <!--Using notNullColumn attribute is required below as LEFT JOIN with empty right side
+    will produce an empty row.-->
+    <collection property="updateEvents"
+                ofType="org.apache.aurora.gen.JobUpdateEvent"
+                columnPrefix="e_"
+                notNullColumn="id">
+      <id column="id" />
+      <result property="status"
+              column="status"
+              typeHandler="org.apache.aurora.scheduler.storage.db.typehandlers.JobUpdateStatusTypeHandler" />
+    </collection>
+    <collection property="instanceEvents"
+                ofType="org.apache.aurora.gen.JobInstanceUpdateEvent"
+                columnPrefix="i_"
+                notNullColumn="id">
+      <id column="id" />
+      <result property="action"
+              column="action"
+              typeHandler="org.apache.aurora.scheduler.storage.db.typehandlers.JobUpdateActionTypeHandler"/>
+    </collection>
+  </resultMap>
+
+  <select id="selectDetails" resultMap="jobUpdateDetailsMap">
+    SELECT
+      u.id AS u_id,
+      u.update_id AS jusm_update_id,
+      u.user AS jusm_user,
+      u.status AS jusm_status,
+      u.created_timestamp_ms AS jusm_created_timestamp_ms,
+      u.last_modified_timestamp_ms AS jusm_last_modified_timestamp_ms,
+      u.update_group_size AS juse_update_group_size,
+      u.max_per_instance_failures AS juse_max_per_instance_failures,
+      u.max_failed_instances AS juse_max_failed_instances,
+      u.max_wait_to_instance_running_ms AS juse_max_wait_to_instance_running_ms,
+      u.min_wait_in_instance_running_ms AS juse_min_wait_in_instance_running_ms,
+      u.rollback_on_failure AS juse_rollback_on_failure,
+      j.id AS jusm_jk_id,
+      j.role AS jusm_jk_role,
+      j.environment AS jusm_jk_environment,
+      j.name AS jusm_jk_name,
+      e.id AS e_id,
+      e.status AS e_status,
+      e.timestamp_ms AS e_timestamp_ms,
+      i.id AS i_id,
+      i.action AS i_action,
+      i.instance_id AS i_instance_id,
+      i.timestamp_ms AS i_timestamp_ms
+    FROM job_updates AS u
+    INNER JOIN job_keys AS j ON j.id = u.job_key_id
+    LEFT OUTER JOIN job_update_events AS e ON e.update_id = u.id
+    LEFT OUTER JOIN job_instance_update_events AS i ON i.update_id = u.id
+    WHERE u.update_id = #{id}
+    ORDER BY e_timestamp_ms, i_timestamp_ms
+  </select>
+
+  <delete id="truncate">
+    DELETE FROM job_updates;
+  </delete>
+</mapper>

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/63792c44/src/main/resources/org/apache/aurora/scheduler/storage/db/JobUpdateEventMapper.xml
----------------------------------------------------------------------
diff --git a/src/main/resources/org/apache/aurora/scheduler/storage/db/JobUpdateEventMapper.xml b/src/main/resources/org/apache/aurora/scheduler/storage/db/JobUpdateEventMapper.xml
new file mode 100644
index 0000000..9b7e8ba
--- /dev/null
+++ b/src/main/resources/org/apache/aurora/scheduler/storage/db/JobUpdateEventMapper.xml
@@ -0,0 +1,35 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ 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.
+ -->
+
+<!DOCTYPE mapper
+        PUBLIC "-//mybatis.org//DTD Mapper 3.0//EN"
+        "http://mybatis.org/dtd/mybatis-3-mapper.dtd">
+<mapper namespace="org.apache.aurora.scheduler.storage.db.JobUpdateEventMapper">
+  <insert id="insert">
+    INSERT INTO job_update_events (
+      update_id,
+      status,
+      timestamp_ms
+    ) VALUES (
+      (
+        SELECT id
+        FROM job_updates
+        WHERE update_id = #{updateId}
+      ),
+      #{event.status, typeHandler=org.apache.aurora.scheduler.storage.db.typehandlers.JobUpdateStatusTypeHandler},
+      #{event.timestampMs}
+    )
+  </insert>
+</mapper>

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/63792c44/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 eb2216a..f0c8336 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
@@ -23,7 +23,7 @@ CREATE TABLE framework_id(
 );
 
 CREATE TABLE job_keys(
-  id INT IDENTITY,
+  id IDENTITY,
   role VARCHAR NOT NULL,
   environment VARCHAR NOT NULL,
   name VARCHAR NOT NULL,
@@ -32,8 +32,8 @@ CREATE TABLE job_keys(
 );
 
 CREATE TABLE locks(
-  id INT IDENTITY,
-  job_key_id INT NOT NULL REFERENCES job_keys(id),
+  id IDENTITY,
+  job_key_id BIGINT NOT NULL REFERENCES job_keys(id),
   token VARCHAR NOT NULL,
   user VARCHAR NOT NULL,
   timestampMs BIGINT NOT NULL,
@@ -43,7 +43,7 @@ CREATE TABLE locks(
 );
 
 CREATE TABLE quotas(
-  id INT IDENTITY,
+  id IDENTITY,
   role VARCHAR NOT NULL,
   num_cpus FLOAT NOT NULL,
   ram_mb INT NOT NULL,
@@ -60,9 +60,9 @@ CREATE TABLE maintenance_modes(
 );
 
 CREATE TABLE host_attributes(
-  id INT IDENTITY,
+  id IDENTITY,
   host VARCHAR NOT NULL,
-  mode TINYINT NOT NULL REFERENCES maintenance_modes(id),
+  mode INT NOT NULL REFERENCES maintenance_modes(id),
   slave_id VARCHAR NOT NULL,
 
   UNIQUE(host),
@@ -70,8 +70,8 @@ CREATE TABLE host_attributes(
 );
 
 CREATE TABLE host_attribute_values(
-  id INT IDENTITY,
-  host_attribute_id INT NOT NULL REFERENCES host_attributes(id)
+  id IDENTITY,
+  host_attribute_id BIGINT NOT NULL REFERENCES host_attributes(id)
   ON DELETE CASCADE,
   name VARCHAR NOT NULL,
   value VARCHAR NOT NULL,
@@ -94,43 +94,41 @@ CREATE TABLE job_update_statuses(
 );
 
 CREATE TABLE job_updates(
-  id INT IDENTITY,
-  job_key_id INT NOT NULL REFERENCES job_keys(id),
+  id IDENTITY,
+  job_key_id BIGINT NOT NULL REFERENCES job_keys(id),
   update_id VARCHAR NOT NULL,
-  user_name VARCHAR NOT NULL,
+  user VARCHAR NOT NULL,
   status INT NOT NULL REFERENCES job_update_statuses(id),
-  inserted_timestamp_ms BIGINT NOT NULL,
-  modified_timestamp_ms BIGINT NOT NULL,
+  created_timestamp_ms BIGINT NOT NULL,
+  last_modified_timestamp_ms BIGINT 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,
-  update_only_these_instances ARRAY,
 
   UNIQUE(update_id)
 );
 
 CREATE TABLE job_update_configs(
-  id INT IDENTITY,
-  update_id INT NOT NULL REFERENCES job_updates(id),
+  id IDENTITY,
+  update_id BIGINT NOT NULL REFERENCES job_updates(id) ON DELETE CASCADE,
   task_config VARCHAR NOT NULL,
-  instances ARRAY NOT NULL,
   is_new BOOLEAN NOT NULL
 );
 
 CREATE TABLE job_update_events(
-  id BIGINT IDENTITY,
-  update_id INT NOT NULL REFERENCES job_updates(id),
-  update_status INT NOT NULL REFERENCES job_update_statuses(id),
+  id IDENTITY,
+  update_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
 );
 
 CREATE TABLE job_instance_update_events(
-  id BIGINT IDENTITY,
-  update_id INT NOT NULL REFERENCES job_updates(id),
-  update_action INT NOT NULL REFERENCES job_instance_update_actions(id),
+  id IDENTITY,
+  update_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/incubator-aurora/blob/63792c44/src/main/thrift/org/apache/aurora/gen/api.thrift
----------------------------------------------------------------------
diff --git a/src/main/thrift/org/apache/aurora/gen/api.thrift b/src/main/thrift/org/apache/aurora/gen/api.thrift
index 35d14d4..4ea9ec2 100644
--- a/src/main/thrift/org/apache/aurora/gen/api.thrift
+++ b/src/main/thrift/org/apache/aurora/gen/api.thrift
@@ -620,29 +620,23 @@ struct JobUpdateSummary {
 
 /** Update configuration and setting details. */
 struct JobUpdateConfiguration {
-  /** Update ID. */
-  1: string updateId
-
   /** Actual InstanceId -> TaskConfig mapping when the update was requested. */
-  2: set<InstanceTaskConfig> oldTaskConfigs
+  1: set<InstanceTaskConfig> oldTaskConfigs
 
   /** Desired InstanceId -> TaskConfig mapping when the update completes. */
-  3: set<InstanceTaskConfig> newTaskConfigs
+  2: set<InstanceTaskConfig> newTaskConfigs
 
   /** Update specific settings. */
-  4: JobUpdateSettings settings
+  3: JobUpdateSettings settings
 }
 
 /** Full definition of the job update. */
 struct JobUpdate {
-  /** Update ID. */
-  1: string updateId
-
   /** Update summary. */
-  2: JobUpdateSummary summary
+  1: JobUpdateSummary summary
 
   /** Update configuration. */
-  3: JobUpdateConfiguration configuration
+  2: JobUpdateConfiguration configuration
 }
 
 struct JobUpdateDetails {

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/63792c44/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
new file mode 100644
index 0000000..c76ab5c
--- /dev/null
+++ b/src/test/java/org/apache/aurora/scheduler/storage/db/DBJobUpdateStoreTest.java
@@ -0,0 +1,308 @@
+/**
+ * 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;
+
+import java.util.List;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableList;
+import com.google.inject.Guice;
+import com.google.inject.Injector;
+import com.twitter.common.inject.Bindings;
+
+import org.apache.aurora.gen.JobInstanceUpdateEvent;
+import org.apache.aurora.gen.JobUpdate;
+import org.apache.aurora.gen.JobUpdateAction;
+import org.apache.aurora.gen.JobUpdateConfiguration;
+import org.apache.aurora.gen.JobUpdateDetails;
+import org.apache.aurora.gen.JobUpdateEvent;
+import org.apache.aurora.gen.JobUpdateSettings;
+import org.apache.aurora.gen.JobUpdateStatus;
+import org.apache.aurora.gen.JobUpdateSummary;
+import org.apache.aurora.scheduler.base.JobKeys;
+import org.apache.aurora.scheduler.storage.Storage;
+import org.apache.aurora.scheduler.storage.Storage.MutableStoreProvider;
+import org.apache.aurora.scheduler.storage.Storage.MutateWork;
+import org.apache.aurora.scheduler.storage.Storage.StorageException;
+import org.apache.aurora.scheduler.storage.Storage.Work.Quiet;
+import org.apache.aurora.scheduler.storage.entities.IJobInstanceUpdateEvent;
+import org.apache.aurora.scheduler.storage.entities.IJobKey;
+import org.apache.aurora.scheduler.storage.entities.IJobUpdate;
+import org.apache.aurora.scheduler.storage.entities.IJobUpdateDetails;
+import org.apache.aurora.scheduler.storage.entities.IJobUpdateEvent;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class DBJobUpdateStoreTest {
+
+  private static final IJobKey JOB = JobKeys.from("testRole", "testEnv", "job");
+
+  private DbStorage storage;
+
+  @Before
+  public void setUp() throws Exception {
+    Injector injector = Guice.createInjector(DbModule.testModule(Bindings.KeyFactory.PLAIN));
+    storage = injector.getInstance(DbStorage.class);
+    storage.prepare();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    truncateUpdates();
+  }
+
+  @Test
+  public void testSaveJobUpdates() {
+    String updateId1 = "u1";
+    String updateId2 = "u2";
+    IJobUpdate update1 = makeJobUpdate(JOB, updateId1, JobUpdateStatus.INIT);
+    IJobUpdate update2 = makeJobUpdate(JOB, updateId2, JobUpdateStatus.INIT);
+
+    saveUpdate(update1);
+    assertEquals(update1, getUpdateDetails(updateId1).get().getUpdate());
+
+    saveUpdate(update2);
+    assertEquals(update2, getUpdateDetails(updateId2).get().getUpdate());
+  }
+
+  @Test
+  public void testSaveJobUpdateTwice() {
+    String updateId = "u1";
+    IJobUpdate update = makeJobUpdate(JOB, updateId, JobUpdateStatus.INIT);
+
+    saveUpdate(update);
+    assertEquals(update, getUpdateDetails(updateId).get().getUpdate());
+
+    JobUpdate builder = update.newBuilder();
+    builder.getSummary().setStatus(JobUpdateStatus.ABORTED);
+    update = IJobUpdate.build(builder);
+    saveUpdate(update);
+    assertEquals(update, getUpdateDetails(updateId).get().getUpdate());
+  }
+
+  @Test
+  public void testSaveJobEvents() {
+    String updateId = "u3";
+    IJobUpdate update = makeJobUpdate(JOB, updateId, JobUpdateStatus.ROLLING_FORWARD);
+    IJobUpdateEvent event1 = makeJobUpdateEvent(JobUpdateStatus.ROLLING_FORWARD, 123L);
+    IJobUpdateEvent event2 = makeJobUpdateEvent(JobUpdateStatus.ROLL_FORWARD_PAUSED, 124L);
+
+    saveUpdate(update);
+    assertEquals(update, getUpdateDetails(updateId).get().getUpdate());
+    assertEquals(0, getUpdateDetails(updateId).get().getUpdateEvents().size());
+
+    saveJobEvent(event2, updateId);
+    assertEquals(event2, getUpdateDetails(updateId).get().getUpdateEvents().get(0));
+
+    saveJobEvent(event1, updateId);
+    assertEquals(event1, getUpdateDetails(updateId).get().getUpdateEvents().get(0));
+    assertEquals(event2, getUpdateDetails(updateId).get().getUpdateEvents().get(1));
+  }
+
+  @Test
+  public void testSaveInstanceEvents() {
+    String updateId = "u3";
+    IJobUpdate update = makeJobUpdate(JOB, updateId, JobUpdateStatus.ROLLING_FORWARD);
+    IJobInstanceUpdateEvent event1 = makeJobInstanceEvent(0, 125L, JobUpdateAction.ADD_INSTANCE);
+    IJobInstanceUpdateEvent event2 = makeJobInstanceEvent(1, 126L, JobUpdateAction.ADD_INSTANCE);
+
+    saveUpdate(update);
+    assertEquals(update, getUpdateDetails(updateId).get().getUpdate());
+    assertEquals(0, getUpdateDetails(updateId).get().getInstanceEvents().size());
+
+    saveJobInstanceEvent(event2, updateId);
+    assertEquals(1, getUpdateDetails(updateId).get().getInstanceEvents().size());
+
+    saveJobInstanceEvent(event1, updateId);
+    assertEquals(2, getUpdateDetails(updateId).get().getInstanceEvents().size());
+
+    // Test event order.
+    assertEquals(event1, getUpdateDetails(updateId).get().getInstanceEvents().get(0));
+    assertEquals(event2, getUpdateDetails(updateId).get().getInstanceEvents().get(1));
+  }
+
+  @Test(expected = StorageException.class)
+  public void testSaveJobEventWithoutUpdateFails() {
+    saveJobEvent(makeJobUpdateEvent(JobUpdateStatus.ROLLING_FORWARD, 123L), "u2");
+  }
+
+  @Test(expected = StorageException.class)
+  public void testSaveInstanceEventWithoutUpdateFails() {
+    saveJobInstanceEvent(makeJobInstanceEvent(0, 125L, JobUpdateAction.ADD_INSTANCE), "u1");
+  }
+
+  @Test
+  public void testMultipleJobDetails() {
+    String updateId1 = "u1";
+    String updateId2 = "u2";
+    IJobUpdateDetails details1 = makeJobDetails(
+        makeJobUpdate(JOB, updateId1, JobUpdateStatus.ABORTED));
+
+    IJobUpdateDetails details2 = makeJobDetails(
+        makeJobUpdate(JOB, updateId2, JobUpdateStatus.ERROR));
+
+    saveUpdate(details1.getUpdate());
+    saveUpdate(details2.getUpdate());
+    assertEquals(details1, getUpdateDetails(updateId1).get());
+    assertEquals(details2, getUpdateDetails(updateId2).get());
+
+    IJobUpdateEvent jEvent11 = makeJobUpdateEvent(JobUpdateStatus.ROLL_BACK_PAUSED, 456L);
+    IJobUpdateEvent jEvent12 = makeJobUpdateEvent(JobUpdateStatus.ERROR, 457L);
+    IJobInstanceUpdateEvent iEvent11 = makeJobInstanceEvent(1, 451L, JobUpdateAction.ADD_INSTANCE);
+    IJobInstanceUpdateEvent iEvent12 = makeJobInstanceEvent(2, 452L, JobUpdateAction.ADD_INSTANCE);
+
+    IJobUpdateEvent jEvent21 = makeJobUpdateEvent(JobUpdateStatus.ROLL_FORWARD_PAUSED, 567L);
+    IJobUpdateEvent jEvent22 = makeJobUpdateEvent(JobUpdateStatus.ABORTED, 568L);
+    IJobInstanceUpdateEvent iEvent21 = makeJobInstanceEvent(3, 561L, JobUpdateAction.ADD_INSTANCE);
+    IJobInstanceUpdateEvent iEvent22 = makeJobInstanceEvent(4, 562L, JobUpdateAction.ADD_INSTANCE);
+
+    details1 = updateJobDetails(
+        details1.getUpdate(),
+        ImmutableList.of(jEvent11, jEvent12), ImmutableList.of(iEvent11, iEvent12));
+
+    details2 = updateJobDetails(
+        details2.getUpdate(),
+        ImmutableList.of(jEvent21, jEvent22), ImmutableList.of(iEvent21, iEvent22));
+
+    saveJobEvent(jEvent11, updateId1);
+    saveJobEvent(jEvent12, updateId1);
+    saveJobInstanceEvent(iEvent11, updateId1);
+    saveJobInstanceEvent(iEvent12, updateId1);
+
+    saveJobEvent(jEvent21, updateId2);
+    saveJobEvent(jEvent22, updateId2);
+    saveJobInstanceEvent(iEvent21, updateId2);
+    saveJobInstanceEvent(iEvent22, updateId2);
+
+    assertEquals(details1, getUpdateDetails(updateId1).get());
+    assertEquals(details2, getUpdateDetails(updateId2).get());
+  }
+
+  @Test
+  public void testTruncateJobUpdates() {
+    String updateId = "u5";
+    IJobUpdate update = makeJobUpdate(JOB, updateId, JobUpdateStatus.INIT);
+    IJobUpdateEvent updateEvent = IJobUpdateEvent.build(
+        new JobUpdateEvent(JobUpdateStatus.ROLLING_FORWARD, 123L));
+    IJobInstanceUpdateEvent instanceEvent = IJobInstanceUpdateEvent.build(
+        new JobInstanceUpdateEvent(0, 125L, JobUpdateAction.ADD_INSTANCE));
+
+    saveUpdate(update);
+    saveJobEvent(updateEvent, updateId);
+    saveJobInstanceEvent(instanceEvent, updateId);
+    assertEquals(update, getUpdateDetails(updateId).get().getUpdate());
+    assertEquals(1, getUpdateDetails(updateId).get().getUpdateEvents().size());
+    assertEquals(1, getUpdateDetails(updateId).get().getInstanceEvents().size());
+
+    truncateUpdates();
+    assertEquals(Optional.<IJobUpdateDetails>absent(), getUpdateDetails(updateId));
+  }
+
+  private Optional<IJobUpdateDetails> getUpdateDetails(final String updateId) {
+    return storage.consistentRead(new Quiet<Optional<IJobUpdateDetails>>() {
+      @Override
+      public Optional<IJobUpdateDetails> apply(Storage.StoreProvider storeProvider) {
+        return storeProvider.getJobUpdateStore().fetchJobUpdateDetails(updateId);
+      }
+    });
+  }
+
+  private void saveUpdate(final IJobUpdate update) {
+    storage.write(new MutateWork.Quiet<Void>() {
+      @Override
+      public Void apply(MutableStoreProvider storeProvider) {
+        storeProvider.getJobUpdateStore().saveJobUpdate(update);
+        return null;
+      }
+    });
+  }
+
+  private void saveJobEvent(final IJobUpdateEvent event, final String updateId) {
+    storage.write(new MutateWork.Quiet<Void>() {
+      @Override
+      public Void apply(MutableStoreProvider storeProvider) {
+        storeProvider.getJobUpdateStore().saveJobUpdateEvent(event, updateId);
+        return null;
+      }
+    });
+  }
+
+  private void saveJobInstanceEvent(final IJobInstanceUpdateEvent event, final String updateId) {
+    storage.write(new MutateWork.Quiet<Void>() {
+      @Override
+      public Void apply(MutableStoreProvider storeProvider) {
+        storeProvider.getJobUpdateStore().saveJobInstanceUpdateEvent(event, updateId);
+        return null;
+      }
+    });
+  }
+
+  private void truncateUpdates() {
+    storage.write(new MutateWork.Quiet<Void>() {
+      @Override
+      public Void apply(MutableStoreProvider storeProvider) {
+        storeProvider.getJobUpdateStore().deleteAllUpdatesAndEvents();
+        return null;
+      }
+    });
+  }
+
+  private IJobUpdateEvent makeJobUpdateEvent(JobUpdateStatus status, long timestampMs) {
+    return IJobUpdateEvent.build(
+        new JobUpdateEvent(status, timestampMs));
+  }
+
+  private IJobInstanceUpdateEvent makeJobInstanceEvent(
+      int instanceId,
+      long timestampMs,
+      JobUpdateAction action) {
+    return IJobInstanceUpdateEvent.build(
+        new JobInstanceUpdateEvent(instanceId, timestampMs, action));
+  }
+
+  private IJobUpdateDetails makeJobDetails(IJobUpdate update) {
+    return updateJobDetails(
+        update,
+        ImmutableList.<IJobUpdateEvent>of(),
+        ImmutableList.<IJobInstanceUpdateEvent>of());
+  }
+
+  private IJobUpdateDetails updateJobDetails(
+      IJobUpdate update,
+      List<IJobUpdateEvent> jobEvents,
+      List<IJobInstanceUpdateEvent> instanceEvents) {
+
+    return IJobUpdateDetails.build(new JobUpdateDetails()
+        .setUpdate(update.newBuilder())
+        .setUpdateEvents(IJobUpdateEvent.toBuildersList(jobEvents))
+        .setInstanceEvents(IJobInstanceUpdateEvent.toBuildersList(instanceEvents)));
+  }
+
+  private IJobUpdate makeJobUpdate(IJobKey jobKey, String updateId, JobUpdateStatus status) {
+    return IJobUpdate.build(new JobUpdate()
+        .setSummary(new JobUpdateSummary()
+            .setUpdateId(updateId)
+            .setJobKey(jobKey.newBuilder())
+            .setUser("user")
+            .setStatus(status)
+            .setCreatedTimestampMs(1223L)
+            .setLastModifiedTimestampMs(1224L))
+        .setConfiguration(new JobUpdateConfiguration()
+            .setSettings(new JobUpdateSettings())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/63792c44/src/test/java/org/apache/aurora/scheduler/storage/log/LogStorageTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/storage/log/LogStorageTest.java b/src/test/java/org/apache/aurora/scheduler/storage/log/LogStorageTest.java
index 8f2be53..548322b 100644
--- a/src/test/java/org/apache/aurora/scheduler/storage/log/LogStorageTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/storage/log/LogStorageTest.java
@@ -752,8 +752,8 @@ public class LogStorageTest extends EasyMockTest {
   @Test
   public void testSaveUpdate() throws Exception {
     final IJobUpdate update = IJobUpdate.build(new JobUpdate()
-        .setUpdateId(UPDATE_ID)
         .setSummary(new JobUpdateSummary()
+            .setUpdateId(UPDATE_ID)
             .setJobKey(JOB_KEY.newBuilder())
             .setStatus(JobUpdateStatus.ROLLED_FORWARD)
             .setUser("user"))

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/63792c44/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplTest.java b/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplTest.java
index 9b5ba5b..cbb9c46 100644
--- a/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplTest.java
@@ -33,6 +33,7 @@ import org.apache.aurora.gen.JobUpdate;
 import org.apache.aurora.gen.JobUpdateDetails;
 import org.apache.aurora.gen.JobUpdateEvent;
 import org.apache.aurora.gen.JobUpdateStatus;
+import org.apache.aurora.gen.JobUpdateSummary;
 import org.apache.aurora.gen.Lock;
 import org.apache.aurora.gen.LockKey;
 import org.apache.aurora.gen.ScheduleStatus;
@@ -101,7 +102,7 @@ public class SnapshotStoreImplTest extends EasyMockTest {
         .setVersion(CURRENT_API_VERSION);
     final String updateId = "updateId";
     IJobUpdateDetails updateDetails = IJobUpdateDetails.build(new JobUpdateDetails()
-        .setUpdate(new JobUpdate().setUpdateId(updateId))
+        .setUpdate(new JobUpdate().setSummary(new JobUpdateSummary().setUpdateId(updateId)))
         .setUpdateEvents(ImmutableList.of(new JobUpdateEvent().setStatus(JobUpdateStatus.INIT)))
         .setInstanceEvents(ImmutableList.of(new JobInstanceUpdateEvent().setTimestampMs(123L))));
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/63792c44/src/test/resources/org/apache/aurora/gen/api.thrift.md5
----------------------------------------------------------------------
diff --git a/src/test/resources/org/apache/aurora/gen/api.thrift.md5 b/src/test/resources/org/apache/aurora/gen/api.thrift.md5
index f01e4b3..6bc6ccf 100644
--- a/src/test/resources/org/apache/aurora/gen/api.thrift.md5
+++ b/src/test/resources/org/apache/aurora/gen/api.thrift.md5
@@ -1 +1 @@
-acef05baef774702d0a4ea294a2df53f
+cea1bcc3b07f8249c10971be8c47671a