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/05 03:13:51 UTC

git commit: Initial implementation of the UpdateStore (saveUpdate).

Repository: incubator-aurora
Updated Branches:
  refs/heads/master 869de2744 -> cccbac85a


Initial implementation of the UpdateStore (saveUpdate).

Bugs closed: AURORA-612

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


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

Branch: refs/heads/master
Commit: cccbac85a4546c18ea3f5057961c97eddf0115b9
Parents: 869de27
Author: Maxim Khutornenko <ma...@apache.org>
Authored: Mon Aug 4 18:13:30 2014 -0700
Committer: Maxim Khutornenko <ma...@apache.org>
Committed: Mon Aug 4 18:13:30 2014 -0700

----------------------------------------------------------------------
 .../scheduler/storage/ForwardingStore.java      | 10 +-
 .../scheduler/storage/JobUpdateStore.java       | 34 +++++++
 .../aurora/scheduler/storage/Storage.java       |  2 +
 .../scheduler/storage/db/DBJobUpdateStore.java  | 27 ++++++
 .../aurora/scheduler/storage/db/DbModule.java   |  2 +
 .../aurora/scheduler/storage/db/DbStorage.java  |  6 ++
 .../scheduler/storage/db/MigrationModule.java   |  2 +
 .../scheduler/storage/log/LogStorage.java       | 21 ++++-
 .../storage/log/WriteAheadStorage.java          | 36 +++++++-
 .../scheduler/storage/mem/MemStorage.java       | 10 +-
 .../thrift/SchedulerThriftInterface.java        | 16 ++--
 .../thrift/org/apache/aurora/gen/api.thrift     | 97 ++++++++++----------
 .../thrift/org/apache/aurora/gen/storage.thrift |  6 ++
 .../scheduler/storage/log/LogStorageTest.java   | 61 ++++++++++--
 .../storage/testing/StorageTestUtil.java        |  3 +
 .../scheduler/thrift/aop/ForwardingThrift.java  | 30 +++---
 .../aurora/client/api/test_scheduler_client.py  | 42 ++++-----
 .../org/apache/aurora/gen/storage.thrift.md5    |  2 +-
 18 files changed, 295 insertions(+), 112 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/cccbac85/src/main/java/org/apache/aurora/scheduler/storage/ForwardingStore.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/ForwardingStore.java b/src/main/java/org/apache/aurora/scheduler/storage/ForwardingStore.java
index be8943d..14f7e99 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/ForwardingStore.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/ForwardingStore.java
@@ -40,7 +40,8 @@ public class ForwardingStore implements
     TaskStore,
     LockStore,
     QuotaStore,
-    AttributeStore {
+    AttributeStore,
+    JobUpdateStore {
 
   private final SchedulerStore schedulerStore;
   private final JobStore jobStore;
@@ -50,7 +51,7 @@ public class ForwardingStore implements
   private final AttributeStore attributeStore;
 
   /**
-   * Creats a new forwarding store that delegates to the providing default stores.
+   * Creates a new forwarding store that delegates to the providing default stores.
    *
    * @param schedulerStore Delegate.
    * @param jobStore Delegate.
@@ -58,6 +59,7 @@ public class ForwardingStore implements
    * @param lockStore Delegate.
    * @param quotaStore Delegate.
    * @param attributeStore Delegate.
+   * @param jobUpdateStore Delegate.
    */
   public ForwardingStore(
       SchedulerStore schedulerStore,
@@ -65,7 +67,8 @@ public class ForwardingStore implements
       TaskStore taskStore,
       LockStore lockStore,
       QuotaStore quotaStore,
-      AttributeStore attributeStore) {
+      AttributeStore attributeStore,
+      JobUpdateStore jobUpdateStore) {
 
     this.schedulerStore = requireNonNull(schedulerStore);
     this.jobStore = requireNonNull(jobStore);
@@ -73,6 +76,7 @@ public class ForwardingStore implements
     this.lockStore = requireNonNull(lockStore);
     this.quotaStore = requireNonNull(quotaStore);
     this.attributeStore = requireNonNull(attributeStore);
+    requireNonNull(jobUpdateStore);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/cccbac85/src/main/java/org/apache/aurora/scheduler/storage/JobUpdateStore.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/JobUpdateStore.java b/src/main/java/org/apache/aurora/scheduler/storage/JobUpdateStore.java
new file mode 100644
index 0000000..1d2e895
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/JobUpdateStore.java
@@ -0,0 +1,34 @@
+/**
+ * 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;
+
+import org.apache.aurora.scheduler.storage.entities.IJobUpdate;
+
+/**
+ * Stores all job updates and defines methods for saving, updating and fetching job updates.
+ */
+public interface JobUpdateStore {
+
+  // TODO(maxim): define get and update event APIs.
+
+  interface Mutable extends JobUpdateStore {
+
+    /**
+     * Saves a new job {@code update}.
+     *
+     * @param update Update to save.
+     */
+    void saveJobUpdate(IJobUpdate update);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/cccbac85/src/main/java/org/apache/aurora/scheduler/storage/Storage.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/Storage.java b/src/main/java/org/apache/aurora/scheduler/storage/Storage.java
index baa5ab6..3268416 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/Storage.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/Storage.java
@@ -37,6 +37,7 @@ public interface Storage {
     LockStore getLockStore();
     QuotaStore getQuotaStore();
     AttributeStore getAttributeStore();
+    JobUpdateStore getUpdateStore();
   }
 
   interface MutableStoreProvider extends StoreProvider {
@@ -61,6 +62,7 @@ public interface Storage {
     LockStore.Mutable getLockStore();
     QuotaStore.Mutable getQuotaStore();
     AttributeStore.Mutable getAttributeStore();
+    JobUpdateStore.Mutable getUpdateStore();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/cccbac85/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
new file mode 100644
index 0000000..e4adfe2
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/DBJobUpdateStore.java
@@ -0,0 +1,27 @@
+/**
+ * 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.scheduler.storage.JobUpdateStore;
+import org.apache.aurora.scheduler.storage.entities.IJobUpdate;
+
+/**
+ * A relational database-backed job update store.
+ */
+public class DBJobUpdateStore implements JobUpdateStore.Mutable {
+  @Override
+  public void saveJobUpdate(IJobUpdate update) {
+    // TODO(maxim): implement DB mapping logic.
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/cccbac85/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 4dbfd8a..5d65218 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
@@ -24,6 +24,7 @@ import com.google.inject.name.Names;
 import com.twitter.common.inject.Bindings;
 
 import org.apache.aurora.scheduler.storage.AttributeStore;
+import org.apache.aurora.scheduler.storage.JobUpdateStore;
 import org.apache.aurora.scheduler.storage.LockStore;
 import org.apache.aurora.scheduler.storage.QuotaStore;
 import org.apache.aurora.scheduler.storage.SchedulerStore;
@@ -131,6 +132,7 @@ public class DbModule extends PrivateModule {
     bindStore(LockStore.Mutable.class, DbLockStore.class);
     bindStore(QuotaStore.Mutable.class, DbQuotaStore.class);
     bindStore(SchedulerStore.Mutable.class, DbSchedulerStore.class);
+    bindStore(JobUpdateStore.Mutable.class, DBJobUpdateStore.class);
 
     Key<Storage> storageKey = keyFactory.create(Storage.class);
     bind(storageKey).to(DbStorage.class);

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/cccbac85/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 0493c5f..43e528f 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
@@ -24,6 +24,7 @@ import com.google.inject.Inject;
 import org.apache.aurora.gen.MaintenanceMode;
 import org.apache.aurora.scheduler.storage.AttributeStore;
 import org.apache.aurora.scheduler.storage.JobStore;
+import org.apache.aurora.scheduler.storage.JobUpdateStore;
 import org.apache.aurora.scheduler.storage.LockStore;
 import org.apache.aurora.scheduler.storage.QuotaStore;
 import org.apache.aurora.scheduler.storage.SchedulerStore;
@@ -108,6 +109,11 @@ class DbStorage extends AbstractIdleService implements Storage {
       public AttributeStore.Mutable getAttributeStore() {
         return attributeStore;
       }
+
+      @Override
+      public JobUpdateStore.Mutable getUpdateStore() {
+        throw new UnsupportedOperationException("Not implemented.");
+      }
     };
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/cccbac85/src/main/java/org/apache/aurora/scheduler/storage/db/MigrationModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/MigrationModule.java b/src/main/java/org/apache/aurora/scheduler/storage/db/MigrationModule.java
index f327873..a821de3 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/db/MigrationModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/MigrationModule.java
@@ -17,6 +17,7 @@ import com.google.inject.AbstractModule;
 import com.twitter.common.inject.Bindings.KeyFactory;
 
 import org.apache.aurora.scheduler.storage.AttributeStore;
+import org.apache.aurora.scheduler.storage.JobUpdateStore;
 import org.apache.aurora.scheduler.storage.LockStore;
 import org.apache.aurora.scheduler.storage.QuotaStore;
 import org.apache.aurora.scheduler.storage.SchedulerStore;
@@ -54,5 +55,6 @@ public class MigrationModule extends AbstractModule {
     link(LockStore.Mutable.class);
     link(QuotaStore.Mutable.class);
     link(SchedulerStore.Mutable.class);
+    link(JobUpdateStore.Mutable.class);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/cccbac85/src/main/java/org/apache/aurora/scheduler/storage/log/LogStorage.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/log/LogStorage.java b/src/main/java/org/apache/aurora/scheduler/storage/log/LogStorage.java
index 67191e7..54d8362 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/log/LogStorage.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/log/LogStorage.java
@@ -49,6 +49,7 @@ import org.apache.aurora.scheduler.log.Log.Stream.StreamAccessException;
 import org.apache.aurora.scheduler.storage.AttributeStore;
 import org.apache.aurora.scheduler.storage.DistributedSnapshotStore;
 import org.apache.aurora.scheduler.storage.JobStore;
+import org.apache.aurora.scheduler.storage.JobUpdateStore;
 import org.apache.aurora.scheduler.storage.LockStore;
 import org.apache.aurora.scheduler.storage.QuotaStore;
 import org.apache.aurora.scheduler.storage.SchedulerStore;
@@ -59,6 +60,7 @@ import org.apache.aurora.scheduler.storage.TaskStore;
 import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 import org.apache.aurora.scheduler.storage.entities.IJobConfiguration;
 import org.apache.aurora.scheduler.storage.entities.IJobKey;
+import org.apache.aurora.scheduler.storage.entities.IJobUpdate;
 import org.apache.aurora.scheduler.storage.entities.ILock;
 import org.apache.aurora.scheduler.storage.entities.ILockKey;
 import org.apache.aurora.scheduler.storage.entities.IResourceAggregate;
@@ -177,6 +179,7 @@ public class LogStorage implements NonVolatileStorage, DistributedSnapshotStore
   private final LockStore.Mutable writeBehindLockStore;
   private final QuotaStore.Mutable writeBehindQuotaStore;
   private final AttributeStore.Mutable writeBehindAttributeStore;
+  private final JobUpdateStore.Mutable writeBehindUpdateStore;
 
   private StreamManager streamManager;
   private final WriteAheadStorage writeAheadStorage;
@@ -226,7 +229,8 @@ public class LogStorage implements NonVolatileStorage, DistributedSnapshotStore
              @WriteBehind TaskStore.Mutable taskStore,
              @WriteBehind LockStore.Mutable lockStore,
              @WriteBehind QuotaStore.Mutable quotaStore,
-             @WriteBehind AttributeStore.Mutable attributeStore) {
+             @WriteBehind AttributeStore.Mutable attributeStore,
+             @WriteBehind JobUpdateStore.Mutable updateStore) {
 
     this(logManager,
         new ScheduledExecutorSchedulingService(shutdownRegistry, shutdownGracePeriod),
@@ -238,7 +242,8 @@ public class LogStorage implements NonVolatileStorage, DistributedSnapshotStore
         taskStore,
         lockStore,
         quotaStore,
-        attributeStore);
+        attributeStore,
+        updateStore);
   }
 
   @VisibleForTesting
@@ -252,7 +257,8 @@ public class LogStorage implements NonVolatileStorage, DistributedSnapshotStore
              TaskStore.Mutable taskStore,
              LockStore.Mutable lockStore,
              QuotaStore.Mutable quotaStore,
-             AttributeStore.Mutable attributeStore) {
+             AttributeStore.Mutable attributeStore,
+             JobUpdateStore.Mutable updateStore) {
 
     this.logManager = requireNonNull(logManager);
     this.schedulingService = requireNonNull(schedulingService);
@@ -270,6 +276,7 @@ public class LogStorage implements NonVolatileStorage, DistributedSnapshotStore
     this.writeBehindLockStore = requireNonNull(lockStore);
     this.writeBehindQuotaStore = requireNonNull(quotaStore);
     this.writeBehindAttributeStore = requireNonNull(attributeStore);
+    this.writeBehindUpdateStore = requireNonNull(updateStore);
     TransactionManager transactionManager = new TransactionManager() {
       @Override
       public boolean hasActiveTransaction() {
@@ -288,7 +295,8 @@ public class LogStorage implements NonVolatileStorage, DistributedSnapshotStore
         taskStore,
         lockStore,
         quotaStore,
-        attributeStore);
+        attributeStore,
+        updateStore);
   }
 
   @Override
@@ -441,6 +449,11 @@ public class LogStorage implements NonVolatileStorage, DistributedSnapshotStore
         writeBehindLockStore.removeLock(ILockKey.build(op.getRemoveLock().getLockKey()));
         break;
 
+      case SAVE_JOB_UPDATE:
+        writeBehindUpdateStore.saveJobUpdate(
+            IJobUpdate.build(op.getSaveJobUpdate().getJobUpdate()));
+        break;
+
       default:
         throw new IllegalStateException("Unknown transaction op: " + op);
     }

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/cccbac85/src/main/java/org/apache/aurora/scheduler/storage/log/WriteAheadStorage.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/log/WriteAheadStorage.java b/src/main/java/org/apache/aurora/scheduler/storage/log/WriteAheadStorage.java
index 414492f..55efa96 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/log/WriteAheadStorage.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/log/WriteAheadStorage.java
@@ -35,6 +35,7 @@ import org.apache.aurora.gen.storage.RewriteTask;
 import org.apache.aurora.gen.storage.SaveAcceptedJob;
 import org.apache.aurora.gen.storage.SaveFrameworkId;
 import org.apache.aurora.gen.storage.SaveHostAttributes;
+import org.apache.aurora.gen.storage.SaveJobUpdate;
 import org.apache.aurora.gen.storage.SaveLock;
 import org.apache.aurora.gen.storage.SaveQuota;
 import org.apache.aurora.gen.storage.SaveTasks;
@@ -43,6 +44,7 @@ import org.apache.aurora.scheduler.base.Tasks;
 import org.apache.aurora.scheduler.storage.AttributeStore;
 import org.apache.aurora.scheduler.storage.ForwardingStore;
 import org.apache.aurora.scheduler.storage.JobStore;
+import org.apache.aurora.scheduler.storage.JobUpdateStore;
 import org.apache.aurora.scheduler.storage.LockStore;
 import org.apache.aurora.scheduler.storage.QuotaStore;
 import org.apache.aurora.scheduler.storage.SchedulerStore;
@@ -51,6 +53,7 @@ import org.apache.aurora.scheduler.storage.TaskStore;
 import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 import org.apache.aurora.scheduler.storage.entities.IJobConfiguration;
 import org.apache.aurora.scheduler.storage.entities.IJobKey;
+import org.apache.aurora.scheduler.storage.entities.IJobUpdate;
 import org.apache.aurora.scheduler.storage.entities.ILock;
 import org.apache.aurora.scheduler.storage.entities.ILockKey;
 import org.apache.aurora.scheduler.storage.entities.IResourceAggregate;
@@ -73,7 +76,8 @@ class WriteAheadStorage extends ForwardingStore implements
     TaskStore.Mutable,
     LockStore.Mutable,
     QuotaStore.Mutable,
-    AttributeStore.Mutable {
+    AttributeStore.Mutable,
+    JobUpdateStore.Mutable {
 
   private static final Logger LOG = Logger.getLogger(WriteAheadStorage.class.getName());
 
@@ -84,6 +88,7 @@ class WriteAheadStorage extends ForwardingStore implements
   private final LockStore.Mutable lockStore;
   private final QuotaStore.Mutable quotaStore;
   private final AttributeStore.Mutable attributeStore;
+  private final JobUpdateStore.Mutable jobUpdateStore;
 
   /**
    * Creates a new write-ahead storage that delegates to the providing default stores.
@@ -95,6 +100,7 @@ class WriteAheadStorage extends ForwardingStore implements
    * @param lockStore      Delegate.
    * @param quotaStore     Delegate.
    * @param attributeStore Delegate.
+   * @param jobUpdateStore Delegate.
    */
   WriteAheadStorage(
       TransactionManager transactionManager,
@@ -103,9 +109,17 @@ class WriteAheadStorage extends ForwardingStore implements
       TaskStore.Mutable taskStore,
       LockStore.Mutable lockStore,
       QuotaStore.Mutable quotaStore,
-      AttributeStore.Mutable attributeStore) {
-
-    super(schedulerStore, jobStore, taskStore, lockStore, quotaStore, attributeStore);
+      AttributeStore.Mutable attributeStore,
+      JobUpdateStore.Mutable jobUpdateStore) {
+
+    super(
+        schedulerStore,
+        jobStore,
+        taskStore,
+        lockStore,
+        quotaStore,
+        attributeStore,
+        jobUpdateStore);
 
     this.transactionManager = requireNonNull(transactionManager);
     this.schedulerStore = requireNonNull(schedulerStore);
@@ -114,6 +128,7 @@ class WriteAheadStorage extends ForwardingStore implements
     this.lockStore = requireNonNull(lockStore);
     this.quotaStore = requireNonNull(quotaStore);
     this.attributeStore = requireNonNull(attributeStore);
+    this.jobUpdateStore = requireNonNull(jobUpdateStore);
   }
 
   private void write(Op op) {
@@ -260,6 +275,14 @@ class WriteAheadStorage extends ForwardingStore implements
   }
 
   @Override
+  public void saveJobUpdate(IJobUpdate update) {
+    requireNonNull(update);
+
+    write(Op.saveJobUpdate(new SaveJobUpdate(update.newBuilder())));
+    jobUpdateStore.saveJobUpdate(update);
+  }
+
+  @Override
   public void deleteAllTasks() {
     throw new UnsupportedOperationException(
         "Unsupported since casual storage users should never be doing this.");
@@ -336,4 +359,9 @@ class WriteAheadStorage extends ForwardingStore implements
   public TaskStore getTaskStore() {
     return this;
   }
+
+  @Override
+  public JobUpdateStore.Mutable getUpdateStore() {
+    return this;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/cccbac85/src/main/java/org/apache/aurora/scheduler/storage/mem/MemStorage.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/mem/MemStorage.java b/src/main/java/org/apache/aurora/scheduler/storage/mem/MemStorage.java
index 0590ea1..f9a1e2a 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/mem/MemStorage.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/mem/MemStorage.java
@@ -34,6 +34,7 @@ import com.twitter.common.stats.Stats;
 
 import org.apache.aurora.scheduler.storage.AttributeStore;
 import org.apache.aurora.scheduler.storage.JobStore;
+import org.apache.aurora.scheduler.storage.JobUpdateStore;
 import org.apache.aurora.scheduler.storage.LockStore;
 import org.apache.aurora.scheduler.storage.QuotaStore;
 import org.apache.aurora.scheduler.storage.ReadWriteLockManager;
@@ -86,7 +87,9 @@ public class MemStorage implements Storage {
       @Delegated final LockStore.Mutable lockStore,
       @Delegated final Storage delegated,
       @Delegated final QuotaStore.Mutable quotaStore,
-      @Delegated final AttributeStore.Mutable attributeStore) {
+      @Delegated final AttributeStore.Mutable attributeStore,
+      @Delegated final JobUpdateStore.Mutable updateStore) {
+
     this.delegatedStore = delegated;
     storeProvider = new MutableStoreProvider() {
       @Override
@@ -123,6 +126,11 @@ public class MemStorage implements Storage {
       public AttributeStore.Mutable getAttributeStore() {
         return attributeStore;
       }
+
+      @Override
+      public JobUpdateStore.Mutable getUpdateStore() {
+        return updateStore;
+      }
     };
 
     Stats.export(new StatImpl<Integer>("storage_lock_threads_waiting") {

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/cccbac85/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java b/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java
index 24bd03f..9e9f397 100644
--- a/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java
+++ b/src/main/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterface.java
@@ -69,6 +69,8 @@ import org.apache.aurora.gen.JobConfiguration;
 import org.apache.aurora.gen.JobKey;
 import org.apache.aurora.gen.JobSummary;
 import org.apache.aurora.gen.JobSummaryResult;
+import org.apache.aurora.gen.JobUpdateQuery;
+import org.apache.aurora.gen.JobUpdateRequest;
 import org.apache.aurora.gen.ListBackupsResult;
 import org.apache.aurora.gen.Lock;
 import org.apache.aurora.gen.LockKey;
@@ -91,8 +93,6 @@ import org.apache.aurora.gen.SessionKey;
 import org.apache.aurora.gen.StartMaintenanceResult;
 import org.apache.aurora.gen.TaskConfig;
 import org.apache.aurora.gen.TaskQuery;
-import org.apache.aurora.gen.UpdateQuery;
-import org.apache.aurora.gen.UpdateRequest;
 import org.apache.aurora.scheduler.base.JobKeys;
 import org.apache.aurora.scheduler.base.Jobs;
 import org.apache.aurora.scheduler.base.Query;
@@ -1247,32 +1247,32 @@ class SchedulerThriftInterface implements AuroraAdmin.Iface {
   }
 
   @Override
-  public Response startUpdate(UpdateRequest request, Lock lock, SessionKey session) {
+  public Response startJobUpdate(JobUpdateRequest request, Lock lock, SessionKey session) {
     throw new UnsupportedOperationException("Not implemented");
   }
 
   @Override
-  public Response pauseUpdate(String updateId, Lock lock, SessionKey session) {
+  public Response pauseJobUpdate(String updateId, Lock lock, SessionKey session) {
     throw new UnsupportedOperationException("Not implemented");
   }
 
   @Override
-  public Response resumeUpdate(String updateId, Lock lock, SessionKey session) {
+  public Response resumeJobUpdate(String updateId, Lock lock, SessionKey session) {
     throw new UnsupportedOperationException("Not implemented");
   }
 
   @Override
-  public Response abortUpdate(String updateId, Lock lock, SessionKey session) {
+  public Response abortJobUpdate(String updateId, Lock lock, SessionKey session) {
     throw new UnsupportedOperationException("Not implemented");
   }
 
   @Override
-  public Response getUpdates(UpdateQuery updateQuery) throws TException {
+  public Response getJobUpdateSummaries(JobUpdateQuery updateQuery) throws TException {
     throw new UnsupportedOperationException("Not implemented");
   }
 
   @Override
-  public Response getUpdateDetails(String updateId) {
+  public Response getJobUpdateDetails(String updateId) {
     throw new UnsupportedOperationException("Not implemented");
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/cccbac85/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 f2b3053..24b065d 100644
--- a/src/main/thrift/org/apache/aurora/gen/api.thrift
+++ b/src/main/thrift/org/apache/aurora/gen/api.thrift
@@ -508,7 +508,7 @@ struct PendingReason {
 }
 
 /** States that a job update may be in. */
-enum UpdateStatus {
+enum JobUpdateStatus {
   /** Update is created but not yet started. */
   INIT = 0,
 
@@ -537,13 +537,13 @@ enum UpdateStatus {
   ERROR = 8
 }
 
-/** Update actions that can be applied to job instances. */
-enum UpdateAction {
+/** Job update actions that can be applied to job instances. */
+enum JobUpdateAction {
   // TODO(maxim): Define when instance update part is completed.
 }
 
 /** Job update thresholds and limits. */
-struct UpdateSettings {
+struct JobUpdateSettings {
   /** Max number of instances being updated at any given moment. */
   1: i32 updateGroupSize
 
@@ -567,24 +567,24 @@ struct UpdateSettings {
 }
 
 /** Event marking a state transition in job update lifecycle. */
-struct UpdateEvent {
+struct JobUpdateEvent {
   /** Update status. */
-  1: UpdateStatus status
+  1: JobUpdateStatus status
 
   /** Epoch timestamp in milliseconds. */
   2: i64 timestampMs
 }
 
 /** Event marking a state transition in job instance update lifecycle. */
-struct InstanceUpdateEvent {
+struct JobInstanceUpdateEvent {
   /** Job instance ID. */
   1: i32 instanceId
 
   /** Epoch timestamp in milliseconds. */
   2: i64 timestampMs
 
-  /** Update action taken on the instance. */
-  3: UpdateAction action
+  /** Job update action taken on the instance. */
+  3: JobUpdateAction action
 }
 
 /** Maps instance IDs to TaskConfigs it. */
@@ -597,7 +597,7 @@ struct InstanceTaskConfig {
 }
 
 /** Job update state. */
-struct Update {
+struct JobUpdateSummary {
   /** Update ID. */
   1: string updateId
 
@@ -608,7 +608,7 @@ struct Update {
   3: string user
 
   /** Current status of the update. */
-  4: UpdateStatus status
+  4: JobUpdateStatus status
 
   /** Creation timestamp in milliseconds. */
   5: i64 createdTimestampMs
@@ -618,7 +618,7 @@ struct Update {
 }
 
 /** Update configuration and setting details. */
-struct UpdateConfiguration {
+struct JobUpdateConfiguration {
   /** Update ID. */
   1: string updateId
 
@@ -629,26 +629,23 @@ struct UpdateConfiguration {
   3: set<InstanceTaskConfig> newTaskConfigs
 
   /** Update specific settings. */
-  4: UpdateSettings settings
+  4: JobUpdateSettings settings
 }
 
-/** Full job update info including all lifecycle events. */
-struct UpdateDetails {
-  /** Current job update state. */
-  1: Update summary
+/** Full definition of the job update. */
+struct JobUpdate {
+  /** Update ID. */
+  1: string updateId
 
-  /** Update configuration and setting details. */
-  2: UpdateConfiguration details
+  /** Update summary. */
+  2: JobUpdateSummary summary
 
-  /** History for this update. */
-  3: list<UpdateEvent> updateEvents
-
-  /** History for the individual instances updated. */
-  4: list<InstanceUpdateEvent> instanceEvents
+  /** Update configuration. */
+  3: JobUpdateConfiguration configuration
 }
 
 /** A request to update the following instances of the existing job. Used by startUpdate. */
-struct UpdateRequest {
+struct JobUpdateRequest {
   /** Job key. */
   1: JobKey jobKey
 
@@ -659,14 +656,14 @@ struct UpdateRequest {
   3: i32 instanceCount
 
   /** Update settings and limits. */
-  4: UpdateSettings settings
+  4: JobUpdateSettings settings
 }
 
 /**
  * Contains a set of restrictions on matching job updates where all restrictions must be met
  * (terms are AND'ed together).
  */
-struct UpdateQuery {
+struct JobUpdateQuery {
   /** Update ID. */
   1: string updateId
 
@@ -677,7 +674,7 @@ struct UpdateQuery {
   3: JobKey jobKey
 
   /** Set of update statuses. */
-  4: set<UpdateStatus> updateStatus
+  4: set<JobUpdateStatus> updateStatus
 
   /** Offset to serve data from. Used by pagination. */
   5: i32 offset
@@ -718,7 +715,6 @@ struct JobSummaryResult {
   1: set<JobSummary> summaries
 }
 
-
 struct GetLocksResult {
   1: set<Lock> locks
 }
@@ -732,18 +728,25 @@ struct GetPendingReasonResult {
 }
 
 /** Result of the startUpdate call. */
-struct StartUpdateResult {
+struct StartJobUpdateResult {
   1: string updateId
 }
 
-/** Result of the getUpdates call. */
-struct GetUpdatesResult {
-  1: set<Update> updates
+/** Result of the getJobUpdateSummaries call. */
+struct GetJobUpdateSummariesResult {
+  1: set<JobUpdateSummary> updateSummaries
 }
 
-/** Result of the getUpdateDetails call. */
-struct GetUpdateDetailsResult {
-  1: UpdateDetails details
+/** Result of the getJobUpdateDetails call. */
+struct GetJobUpdateDetailsResult {
+  /** Update definition. */
+  1: JobUpdate update
+
+  /** History for this update. */
+  2: list<JobUpdateEvent> updateEvents
+
+  /** History for the individual instances updated. */
+  3: list<JobInstanceUpdateEvent> instanceEvents
 }
 
 /** Information about the scheduler. */
@@ -772,9 +775,9 @@ union Result {
   19: GetLocksResult getLocksResult
   20: ConfigSummaryResult configSummaryResult
   21: GetPendingReasonResult getPendingReasonResult
-  22: StartUpdateResult startUpdateResult
-  23: GetUpdatesResult getUpdatesResult
-  24: GetUpdateDetailsResult getUpdateDetailsResult
+  22: StartJobUpdateResult startJobUpdateResult
+  23: GetJobUpdateSummariesResult getJobUpdateSummariesResult
+  24: GetJobUpdateDetailsResult getJobUpdateDetailsResult
 }
 
 struct ResponseDetail {
@@ -845,11 +848,11 @@ service ReadOnlyScheduler {
   /** Returns all stored context specific resource/operation locks. */
   Response getLocks()
 
-  /** Gets job updates. Not implemented yet. */
-  Response getUpdates(1: UpdateQuery updateQuery)
+  /** Gets job update summaries. Not implemented yet. */
+  Response getJobUpdateSummaries(1: JobUpdateQuery jobUpdateQuery)
 
   /** Gets job update details. Not implemented yet. */
-  Response getUpdateDetails(1: string updateId)
+  Response getJobUpdateDetails(1: string updateId)
 }
 
 // Due to assumptions in the client all authenticated RPCs must have a SessionKey as their
@@ -912,19 +915,19 @@ service AuroraSchedulerManager extends ReadOnlyScheduler {
   Response replaceCronTemplate(1: JobConfiguration config, 2: Lock lock, 3: SessionKey session)
 
   /** Starts update of the existing service job. Not implemented yet. */
-  Response startUpdate(
-      1: UpdateRequest request,
+  Response startJobUpdate(
+      1: JobUpdateRequest request,
       2: Lock lock,
       3: SessionKey session)
 
   /** Pauses the update progress. Can be resumed by resumeUpdate call. Not implemented yet. */
-  Response pauseUpdate(1: string updateId, 2: Lock lock, 3: SessionKey session)
+  Response pauseJobUpdate(1: string updateId, 2: Lock lock, 3: SessionKey session)
 
   /** Resumes progress of a previously paused update. Not implemented yet. */
-  Response resumeUpdate(1: string updateId, 2: Lock lock, 3: SessionKey session)
+  Response resumeJobUpdate(1: string updateId, 2: Lock lock, 3: SessionKey session)
 
   /** Permanently aborts the update. Does not remove the update history. Not implemented yet. */
-  Response abortUpdate(1: string updateId, 2: Lock lock, 3: SessionKey session)
+  Response abortJobUpdate(1: string updateId, 2: Lock lock, 3: SessionKey session)
 }
 
 struct InstanceConfigRewrite {

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/cccbac85/src/main/thrift/org/apache/aurora/gen/storage.thrift
----------------------------------------------------------------------
diff --git a/src/main/thrift/org/apache/aurora/gen/storage.thrift b/src/main/thrift/org/apache/aurora/gen/storage.thrift
index 927b6db..816086e 100644
--- a/src/main/thrift/org/apache/aurora/gen/storage.thrift
+++ b/src/main/thrift/org/apache/aurora/gen/storage.thrift
@@ -63,6 +63,10 @@ struct RemoveQuota {
   1: string role
 }
 
+struct SaveJobUpdate {
+  1: api.JobUpdate jobUpdate
+}
+
 struct SaveHostAttributes {
   1: api.HostAttributes hostAttributes
 }
@@ -79,6 +83,7 @@ union Op {
   11: RewriteTask rewriteTask
   12: SaveLock saveLock
   13: RemoveLock removeLock
+  14: SaveJobUpdate saveJobUpdate
 }
 
 // The current schema version ID.  This should be incremented each time the
@@ -130,6 +135,7 @@ struct Snapshot {
   6: SchedulerMetadata schedulerMetadata
   8: set<QuotaConfiguration> quotaConfigurations
   9: set<api.Lock> locks
+  10: set<api.JobUpdate> jobUpdates
 }
 
 // A message header that calls out the number of expected FrameChunks to follow to form a complete

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/cccbac85/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 9ec00bb..bef0ee7 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
@@ -33,10 +33,16 @@ import org.apache.aurora.codec.ThriftBinaryCodec.CodingException;
 import org.apache.aurora.gen.AssignedTask;
 import org.apache.aurora.gen.Attribute;
 import org.apache.aurora.gen.HostAttributes;
+import org.apache.aurora.gen.InstanceTaskConfig;
 import org.apache.aurora.gen.JobConfiguration;
-import org.apache.aurora.gen.JobKey;
+import org.apache.aurora.gen.JobUpdate;
+import org.apache.aurora.gen.JobUpdateConfiguration;
+import org.apache.aurora.gen.JobUpdateSettings;
+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.Range;
 import org.apache.aurora.gen.ResourceAggregate;
 import org.apache.aurora.gen.ScheduleStatus;
 import org.apache.aurora.gen.ScheduledTask;
@@ -51,6 +57,7 @@ import org.apache.aurora.gen.storage.RewriteTask;
 import org.apache.aurora.gen.storage.SaveAcceptedJob;
 import org.apache.aurora.gen.storage.SaveFrameworkId;
 import org.apache.aurora.gen.storage.SaveHostAttributes;
+import org.apache.aurora.gen.storage.SaveJobUpdate;
 import org.apache.aurora.gen.storage.SaveLock;
 import org.apache.aurora.gen.storage.SaveQuota;
 import org.apache.aurora.gen.storage.SaveTasks;
@@ -73,6 +80,7 @@ import org.apache.aurora.scheduler.storage.Storage.MutateWork.NoResult;
 import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 import org.apache.aurora.scheduler.storage.entities.IJobConfiguration;
 import org.apache.aurora.scheduler.storage.entities.IJobKey;
+import org.apache.aurora.scheduler.storage.entities.IJobUpdate;
 import org.apache.aurora.scheduler.storage.entities.ILock;
 import org.apache.aurora.scheduler.storage.entities.ILockKey;
 import org.apache.aurora.scheduler.storage.entities.IResourceAggregate;
@@ -98,6 +106,7 @@ import static org.junit.Assert.assertTrue;
 public class LogStorageTest extends EasyMockTest {
 
   private static final Amount<Long, Time> SNAPSHOT_INTERVAL = Amount.of(1L, Time.MINUTES);
+  private static final IJobKey JOB_KEY = JobKeys.from("role", "env", "name");
   private static final long NOW = 42L;
 
   private LogStorage logStorage;
@@ -130,7 +139,8 @@ public class LogStorageTest extends EasyMockTest {
             storageUtil.taskStore,
             storageUtil.lockStore,
             storageUtil.quotaStore,
-            storageUtil.attributeStore);
+            storageUtil.attributeStore,
+            storageUtil.updateStore);
 
     stream = createMock(Stream.class);
     streamMatcher = LogOpMatcher.matcherFor(stream);
@@ -344,7 +354,7 @@ public class LogStorageTest extends EasyMockTest {
   @Test
   public void testSaveAcceptedJob() throws Exception {
     final IJobConfiguration jobConfig =
-        IJobConfiguration.build(new JobConfiguration().setKey(new JobKey("owner", "env", "jake")));
+        IJobConfiguration.build(new JobConfiguration().setKey(JOB_KEY.newBuilder()));
     final String managerId = "CRON";
     new MutationFixture() {
       @Override
@@ -365,20 +375,19 @@ public class LogStorageTest extends EasyMockTest {
 
   @Test
   public void testRemoveJob() throws Exception {
-    final IJobKey jobKey = JobKeys.from("role", "env", "name");
     new MutationFixture() {
       @Override
       protected void setupExpectations() throws Exception {
         storageUtil.expectWriteOperation();
-        storageUtil.jobStore.removeJob(jobKey);
+        storageUtil.jobStore.removeJob(JOB_KEY);
         streamMatcher.expectTransaction(
-            Op.removeJob(new RemoveJob().setJobKey(jobKey.newBuilder())))
+            Op.removeJob(new RemoveJob().setJobKey(JOB_KEY.newBuilder())))
             .andReturn(position);
       }
 
       @Override
       protected void performMutations(MutableStoreProvider storeProvider) {
-        storeProvider.getJobStore().removeJob(jobKey);
+        storeProvider.getJobStore().removeJob(JOB_KEY);
       }
     }.run();
   }
@@ -633,7 +642,7 @@ public class LogStorageTest extends EasyMockTest {
   @Test
   public void testSaveLock() throws Exception {
     final ILock lock = ILock.build(new Lock()
-        .setKey(LockKey.job(JobKeys.from("testRole", "testEnv", "testJob").newBuilder()))
+        .setKey(LockKey.job(JOB_KEY.newBuilder()))
         .setToken("testLockId")
         .setUser("testUser")
         .setTimestampMs(12345L));
@@ -656,7 +665,7 @@ public class LogStorageTest extends EasyMockTest {
   @Test
   public void testRemoveLock() throws Exception {
     final ILockKey lockKey =
-        ILockKey.build(LockKey.job(JobKeys.from("testRole", "testEnv", "testJob").newBuilder()));
+        ILockKey.build(LockKey.job(JOB_KEY.newBuilder()));
     new MutationFixture() {
       @Override
       protected void setupExpectations() throws Exception {
@@ -715,6 +724,40 @@ public class LogStorageTest extends EasyMockTest {
     }.run();
   }
 
+  @Test
+  public void testSaveUpdate() throws Exception {
+    final String updateId = "testUpdateId";
+    final IJobUpdate update = IJobUpdate.build(new JobUpdate()
+        .setUpdateId(updateId)
+        .setSummary(new JobUpdateSummary()
+            .setJobKey(JOB_KEY.newBuilder())
+            .setStatus(JobUpdateStatus.ROLLED_FORWARD)
+            .setUser("user"))
+        .setConfiguration(new JobUpdateConfiguration()
+            .setNewTaskConfigs(ImmutableSet.of(new InstanceTaskConfig()
+                .setTask(new TaskConfig())
+                .setInstances(ImmutableList.of(new Range(0, 3)))))
+            .setOldTaskConfigs(ImmutableSet.of(new InstanceTaskConfig()
+                .setTask(new TaskConfig())
+                .setInstances(ImmutableList.of(new Range(0, 3)))))
+            .setSettings(new JobUpdateSettings())));
+
+    new MutationFixture() {
+      @Override
+      protected void setupExpectations() throws Exception {
+        storageUtil.expectWriteOperation();
+        storageUtil.updateStore.saveJobUpdate(update);
+        streamMatcher.expectTransaction(Op.saveJobUpdate(new SaveJobUpdate(update.newBuilder())))
+            .andReturn(position);
+      }
+
+      @Override
+      protected void performMutations(MutableStoreProvider storeProvider) {
+        storeProvider.getUpdateStore().saveJobUpdate(update);
+      }
+    }.run();
+  }
+
   private LogEntry createTransaction(Op... ops) {
     return LogEntry.transaction(
         new Transaction(ImmutableList.copyOf(ops), storageConstants.CURRENT_SCHEMA_VERSION));

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/cccbac85/src/test/java/org/apache/aurora/scheduler/storage/testing/StorageTestUtil.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/storage/testing/StorageTestUtil.java b/src/test/java/org/apache/aurora/scheduler/storage/testing/StorageTestUtil.java
index defafb6..fda1978 100644
--- a/src/test/java/org/apache/aurora/scheduler/storage/testing/StorageTestUtil.java
+++ b/src/test/java/org/apache/aurora/scheduler/storage/testing/StorageTestUtil.java
@@ -20,6 +20,7 @@ import com.twitter.common.testing.easymock.EasyMockTest;
 import org.apache.aurora.scheduler.base.Query;
 import org.apache.aurora.scheduler.storage.AttributeStore;
 import org.apache.aurora.scheduler.storage.JobStore;
+import org.apache.aurora.scheduler.storage.JobUpdateStore;
 import org.apache.aurora.scheduler.storage.LockStore;
 import org.apache.aurora.scheduler.storage.QuotaStore;
 import org.apache.aurora.scheduler.storage.SchedulerStore;
@@ -53,6 +54,7 @@ public class StorageTestUtil {
   public final JobStore.Mutable jobStore;
   public final LockStore.Mutable lockStore;
   public final SchedulerStore.Mutable schedulerStore;
+  public final JobUpdateStore.Mutable updateStore;
   public final NonVolatileStorage storage;
 
   /**
@@ -69,6 +71,7 @@ public class StorageTestUtil {
     this.jobStore = easyMock.createMock(JobStore.Mutable.class);
     this.lockStore = easyMock.createMock(LockStore.Mutable.class);
     this.schedulerStore = easyMock.createMock(SchedulerStore.Mutable.class);
+    this.updateStore = easyMock.createMock(JobUpdateStore.Mutable.class);
     this.storage = easyMock.createMock(NonVolatileStorage.class);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/cccbac85/src/test/java/org/apache/aurora/scheduler/thrift/aop/ForwardingThrift.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/thrift/aop/ForwardingThrift.java b/src/test/java/org/apache/aurora/scheduler/thrift/aop/ForwardingThrift.java
index 5c51175..38bc9ed 100644
--- a/src/test/java/org/apache/aurora/scheduler/thrift/aop/ForwardingThrift.java
+++ b/src/test/java/org/apache/aurora/scheduler/thrift/aop/ForwardingThrift.java
@@ -20,6 +20,8 @@ import org.apache.aurora.gen.AuroraAdmin;
 import org.apache.aurora.gen.Hosts;
 import org.apache.aurora.gen.JobConfiguration;
 import org.apache.aurora.gen.JobKey;
+import org.apache.aurora.gen.JobUpdateQuery;
+import org.apache.aurora.gen.JobUpdateRequest;
 import org.apache.aurora.gen.Lock;
 import org.apache.aurora.gen.LockKey;
 import org.apache.aurora.gen.LockValidation;
@@ -29,8 +31,6 @@ import org.apache.aurora.gen.RewriteConfigsRequest;
 import org.apache.aurora.gen.ScheduleStatus;
 import org.apache.aurora.gen.SessionKey;
 import org.apache.aurora.gen.TaskQuery;
-import org.apache.aurora.gen.UpdateQuery;
-import org.apache.aurora.gen.UpdateRequest;
 import org.apache.thrift.TException;
 
 import static java.util.Objects.requireNonNull;
@@ -266,34 +266,36 @@ abstract class ForwardingThrift implements AuroraAdmin.Iface {
   }
 
   @Override
-  public Response startUpdate(UpdateRequest request, Lock lock, SessionKey session)
+  public Response startJobUpdate(JobUpdateRequest request, Lock lock, SessionKey session)
       throws TException {
 
-    return delegate.startUpdate(request, lock, session);
+    return delegate.startJobUpdate(request, lock, session);
   }
 
   @Override
-  public Response pauseUpdate(String updateId, Lock lock, SessionKey session) throws TException {
-    return delegate.pauseUpdate(updateId, lock, session);
+  public Response pauseJobUpdate(String updateId, Lock lock, SessionKey session) throws TException {
+    return delegate.pauseJobUpdate(updateId, lock, session);
   }
 
   @Override
-  public Response resumeUpdate(String updateId, Lock lock, SessionKey session) throws TException {
-    return delegate.resumeUpdate(updateId, lock, session);
+  public Response resumeJobUpdate(String updateId, Lock lock, SessionKey session)
+      throws TException {
+
+    return delegate.resumeJobUpdate(updateId, lock, session);
   }
 
   @Override
-  public Response abortUpdate(String updateId, Lock lock, SessionKey session) throws TException {
-    return delegate.abortUpdate(updateId, lock, session);
+  public Response abortJobUpdate(String updateId, Lock lock, SessionKey session) throws TException {
+    return delegate.abortJobUpdate(updateId, lock, session);
   }
 
   @Override
-  public Response getUpdates(UpdateQuery updateQuery) throws TException {
-    return delegate.getUpdates(updateQuery);
+  public Response getJobUpdateSummaries(JobUpdateQuery updateQuery) throws TException {
+    return delegate.getJobUpdateSummaries(updateQuery);
   }
 
   @Override
-  public Response getUpdateDetails(String updateId) throws TException {
-    return delegate.getUpdateDetails(updateId);
+  public Response getJobUpdateDetails(String updateId) throws TException {
+    return delegate.getJobUpdateDetails(updateId);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/cccbac85/src/test/python/apache/aurora/client/api/test_scheduler_client.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/api/test_scheduler_client.py b/src/test/python/apache/aurora/client/api/test_scheduler_client.py
index 9721ece..ab74db3 100644
--- a/src/test/python/apache/aurora/client/api/test_scheduler_client.py
+++ b/src/test/python/apache/aurora/client/api/test_scheduler_client.py
@@ -35,6 +35,8 @@ from gen.apache.aurora.api.ttypes import (
     Hosts,
     JobConfiguration,
     JobKey,
+    JobUpdateQuery,
+    JobUpdateRequest,
     Lock,
     LockValidation,
     ResourceAggregate,
@@ -44,9 +46,7 @@ from gen.apache.aurora.api.ttypes import (
     RewriteConfigsRequest,
     ScheduleStatus,
     SessionKey,
-    TaskQuery,
-    UpdateQuery,
-    UpdateRequest
+    TaskQuery
 )
 
 ROLE = 'foorole'
@@ -252,35 +252,35 @@ class TestSchedulerProxyAdminInjection(TestSchedulerProxyInjection):
     self.mox.ReplayAll()
     self.make_scheduler_proxy().rewriteConfigs(RewriteConfigsRequest())
 
-  def test_getUpdates(self):
-    self.mock_thrift_client.getUpdates(IsA(UpdateQuery))
+  def test_getJobUpdateSummaries(self):
+    self.mock_thrift_client.getJobUpdateSummaries(IsA(JobUpdateQuery))
     self.mox.ReplayAll()
-    self.make_scheduler_proxy().getUpdates(UpdateQuery())
+    self.make_scheduler_proxy().getJobUpdateSummaries(JobUpdateQuery())
 
-  def test_getUpdateDetails(self):
-    self.mock_thrift_client.getUpdateDetails('update_id')
+  def test_getJobUpdateDetails(self):
+    self.mock_thrift_client.getJobUpdateDetails('update_id')
     self.mox.ReplayAll()
-    self.make_scheduler_proxy().getUpdateDetails('update_id')
+    self.make_scheduler_proxy().getJobUpdateDetails('update_id')
 
-  def test_startUpdate(self):
-    self.mock_thrift_client.startUpdate(IsA(UpdateRequest), IsA(Lock), IsA(SessionKey))
+  def test_startJobUpdate(self):
+    self.mock_thrift_client.startJobUpdate(IsA(JobUpdateRequest), IsA(Lock), IsA(SessionKey))
     self.mox.ReplayAll()
-    self.make_scheduler_proxy().startUpdate(UpdateRequest(), Lock())
+    self.make_scheduler_proxy().startJobUpdate(JobUpdateRequest(), Lock())
 
-  def test_pauseUpdate(self):
-    self.mock_thrift_client.pauseUpdate('update_id', IsA(Lock), IsA(SessionKey))
+  def test_pauseJobUpdate(self):
+    self.mock_thrift_client.pauseJobUpdate('update_id', IsA(Lock), IsA(SessionKey))
     self.mox.ReplayAll()
-    self.make_scheduler_proxy().pauseUpdate('update_id', Lock())
+    self.make_scheduler_proxy().pauseJobUpdate('update_id', Lock())
 
-  def test_resumeUpdate(self):
-    self.mock_thrift_client.resumeUpdate('update_id', IsA(Lock), IsA(SessionKey))
+  def test_resumeJobUpdate(self):
+    self.mock_thrift_client.resumeJobUpdate('update_id', IsA(Lock), IsA(SessionKey))
     self.mox.ReplayAll()
-    self.make_scheduler_proxy().resumeUpdate('update_id', Lock())
+    self.make_scheduler_proxy().resumeJobUpdate('update_id', Lock())
 
-  def test_abortUpdate(self):
-    self.mock_thrift_client.abortUpdate('update_id', IsA(Lock), IsA(SessionKey))
+  def test_abortJobUpdate(self):
+    self.mock_thrift_client.abortJobUpdate('update_id', IsA(Lock), IsA(SessionKey))
     self.mox.ReplayAll()
-    self.make_scheduler_proxy().abortUpdate('update_id', Lock())
+    self.make_scheduler_proxy().abortJobUpdate('update_id', Lock())
 
 
 @pytest.mark.parametrize('scheme', ('http', 'https'))

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/cccbac85/src/test/resources/org/apache/aurora/gen/storage.thrift.md5
----------------------------------------------------------------------
diff --git a/src/test/resources/org/apache/aurora/gen/storage.thrift.md5 b/src/test/resources/org/apache/aurora/gen/storage.thrift.md5
index 61f426b..a93291e 100644
--- a/src/test/resources/org/apache/aurora/gen/storage.thrift.md5
+++ b/src/test/resources/org/apache/aurora/gen/storage.thrift.md5
@@ -1 +1 @@
-6aa8d645049f6850d4a326a9a30645a0
+e5bfb3fa16323868992dc90a90fffcc0