Repository: aurora
Updated Branches:
  refs/heads/master fb64df240 -> 4fecf1f59


http://git-wip-us.apache.org/repos/asf/aurora/blob/4fecf1f5/src/test/java/org/apache/aurora/scheduler/storage/AbstractJobUpdateStoreTest.java
----------------------------------------------------------------------
diff --git 
a/src/test/java/org/apache/aurora/scheduler/storage/AbstractJobUpdateStoreTest.java
 
b/src/test/java/org/apache/aurora/scheduler/storage/AbstractJobUpdateStoreTest.java
index 5c82bcf..9fab7ae 100644
--- 
a/src/test/java/org/apache/aurora/scheduler/storage/AbstractJobUpdateStoreTest.java
+++ 
b/src/test/java/org/apache/aurora/scheduler/storage/AbstractJobUpdateStoreTest.java
@@ -21,7 +21,6 @@ import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
 import com.google.common.base.Optional;
-import com.google.common.collect.FluentIterable;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
@@ -41,12 +40,9 @@ import org.apache.aurora.gen.JobUpdateSettings;
 import org.apache.aurora.gen.JobUpdateState;
 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.Metadata;
 import org.apache.aurora.gen.Range;
 import org.apache.aurora.gen.TaskConfig;
-import org.apache.aurora.gen.storage.StoredJobUpdateDetails;
 import org.apache.aurora.scheduler.base.JobKeys;
 import org.apache.aurora.scheduler.base.TaskTestUtil;
 import org.apache.aurora.scheduler.storage.Storage.MutateWork.NoResult;
@@ -61,8 +57,6 @@ import 
org.apache.aurora.scheduler.storage.entities.IJobUpdateKey;
 import org.apache.aurora.scheduler.storage.entities.IJobUpdateQuery;
 import org.apache.aurora.scheduler.storage.entities.IJobUpdateState;
 import org.apache.aurora.scheduler.storage.entities.IJobUpdateSummary;
-import org.apache.aurora.scheduler.storage.entities.ILock;
-import org.apache.aurora.scheduler.storage.entities.ILockKey;
 import org.apache.aurora.scheduler.storage.testing.StorageEntityUtil;
 import org.apache.aurora.scheduler.testing.FakeStatsProvider;
 import org.junit.After;
@@ -156,16 +150,16 @@ public abstract class AbstractJobUpdateStoreTest {
         StorageEntityUtil.getField(IJobUpdateSummary.class, "state"),
         StorageEntityUtil.getField(Range.class, "first"),
         StorageEntityUtil.getField(Range.class, "last"));
-    saveUpdate(update1, Optional.of("lock1"));
+    saveUpdate(update1);
     assertUpdate(update1);
 
-    saveUpdate(update2, Optional.absent());
+    saveUpdate(update2);
     assertUpdate(update2);
 
     // Colliding update keys should be forbidden.
     IJobUpdate update3 = makeJobUpdate(updateId2);
     try {
-      saveUpdate(update3, Optional.absent());
+      saveUpdate(update3);
       fail("Update ID collision should not be allowed");
     } catch (StorageException e) {
       // Expected.
@@ -194,7 +188,7 @@ public abstract class AbstractJobUpdateStoreTest {
         StorageEntityUtil.getField(IJobUpdateSummary.class, "state"),
         StorageEntityUtil.getField(Range.class, "first"),
         StorageEntityUtil.getField(Range.class, "last"));
-    saveUpdate(update, Optional.of("lock1"));
+    saveUpdate(update);
     assertUpdate(update);
   }
 
@@ -204,7 +198,7 @@ public abstract class AbstractJobUpdateStoreTest {
     builder.getInstructions().unsetInitialState();
 
     // Save with null initial state instances.
-    saveUpdate(IJobUpdate.build(builder), Optional.of("lock"));
+    saveUpdate(IJobUpdate.build(builder));
 
     builder.getInstructions().setInitialState(ImmutableSet.of());
     assertUpdate(IJobUpdate.build(builder));
@@ -216,7 +210,7 @@ public abstract class AbstractJobUpdateStoreTest {
     builder.getInstructions().unsetDesiredState();
 
     // Save with null desired state instances.
-    saveUpdate(IJobUpdate.build(builder), Optional.of("lock"));
+    saveUpdate(IJobUpdate.build(builder));
 
     assertUpdate(IJobUpdate.build(builder));
   }
@@ -227,7 +221,7 @@ public abstract class AbstractJobUpdateStoreTest {
     builder.getInstructions().unsetInitialState();
     builder.getInstructions().unsetDesiredState();
 
-    saveUpdate(IJobUpdate.build(builder), Optional.of("lock"));
+    saveUpdate(IJobUpdate.build(builder));
   }
 
   @Test(expected = NullPointerException.class)
@@ -236,7 +230,7 @@ public abstract class AbstractJobUpdateStoreTest {
     builder.getInstructions().getInitialState().add(
         new InstanceTaskConfig(null, ImmutableSet.of()));
 
-    saveUpdate(IJobUpdate.build(builder), Optional.of("lock"));
+    saveUpdate(IJobUpdate.build(builder));
   }
 
   @Test(expected = IllegalArgumentException.class)
@@ -247,7 +241,7 @@ public abstract class AbstractJobUpdateStoreTest {
             TaskTestUtil.makeConfig(TaskTestUtil.JOB).newBuilder(),
             ImmutableSet.of()));
 
-    saveUpdate(IJobUpdate.build(builder), Optional.of("lock"));
+    saveUpdate(IJobUpdate.build(builder));
   }
 
   @Test(expected = NullPointerException.class)
@@ -255,7 +249,7 @@ public abstract class AbstractJobUpdateStoreTest {
     JobUpdate builder = makeJobUpdate(makeKey("u1")).newBuilder();
     builder.getInstructions().getDesiredState().setTask(null);
 
-    saveUpdate(IJobUpdate.build(builder), Optional.of("lock"));
+    saveUpdate(IJobUpdate.build(builder));
   }
 
   @Test(expected = IllegalArgumentException.class)
@@ -263,7 +257,7 @@ public abstract class AbstractJobUpdateStoreTest {
     JobUpdate builder = makeJobUpdate(makeKey("u1")).newBuilder();
     
builder.getInstructions().getDesiredState().setInstances(ImmutableSet.of());
 
-    saveUpdate(IJobUpdate.build(builder), Optional.of("lock"));
+    saveUpdate(IJobUpdate.build(builder));
   }
 
   @Test
@@ -277,7 +271,7 @@ public abstract class AbstractJobUpdateStoreTest {
     IJobUpdate expected = IJobUpdate.build(builder);
 
     // Save with empty overrides.
-    saveUpdate(expected, Optional.of("lock"));
+    saveUpdate(expected);
     assertUpdate(expected);
   }
 
@@ -293,7 +287,7 @@ public abstract class AbstractJobUpdateStoreTest {
 
     // Save with null overrides.
     builder.getInstructions().getSettings().setUpdateOnlyTheseInstances(null);
-    saveUpdate(IJobUpdate.build(builder), Optional.of("lock"));
+    saveUpdate(IJobUpdate.build(builder));
     assertUpdate(expected);
   }
 
@@ -302,8 +296,8 @@ public abstract class AbstractJobUpdateStoreTest {
     IJobUpdateKey updateId = makeKey("u1");
     IJobUpdate update = makeJobUpdate(updateId);
 
-    saveUpdate(update, Optional.of("lock1"));
-    saveUpdate(update, Optional.of("lock2"));
+    saveUpdate(update);
+    saveUpdate(update);
   }
 
   @Test
@@ -313,7 +307,7 @@ public abstract class AbstractJobUpdateStoreTest {
     IJobUpdateEvent event1 = makeJobUpdateEvent(ROLLING_FORWARD, 124L);
     IJobUpdateEvent event2 = makeJobUpdateEvent(ROLL_FORWARD_PAUSED, 125L);
 
-    saveUpdate(update, Optional.of("lock1"));
+    saveUpdate(update);
     assertUpdate(update);
     assertEquals(ImmutableList.of(FIRST_EVENT), 
getUpdateDetails(updateId).get().getUpdateEvents());
 
@@ -347,7 +341,7 @@ public abstract class AbstractJobUpdateStoreTest {
     IJobInstanceUpdateEvent event1 = makeJobInstanceEvent(0, 125L, 
INSTANCE_UPDATED);
     IJobInstanceUpdateEvent event2 = makeJobInstanceEvent(1, 126L, 
INSTANCE_ROLLING_BACK);
 
-    saveUpdate(update, Optional.of("lock"));
+    saveUpdate(update);
     assertUpdate(update);
     assertEquals(0, 
getUpdateDetails(updateId).get().getInstanceEvents().size());
 
@@ -383,7 +377,7 @@ public abstract class AbstractJobUpdateStoreTest {
   public void testSaveJobUpdateStateIgnored() {
     IJobUpdateKey updateId = makeKey("u1");
     IJobUpdate update = populateExpected(makeJobUpdate(updateId), ABORTED, 
567L, 567L);
-    saveUpdate(update, Optional.of("lock1"));
+    saveUpdate(update);
 
     // Assert state fields were ignored.
     assertUpdate(update);
@@ -398,8 +392,8 @@ public abstract class AbstractJobUpdateStoreTest {
 
     assertEquals(ImmutableList.of(), getInstanceEvents(updateId2, 3));
 
-    saveUpdate(details1.getUpdate(), Optional.of("lock1"));
-    saveUpdate(details2.getUpdate(), Optional.of("lock2"));
+    saveUpdate(details1.getUpdate());
+    saveUpdate(details2.getUpdate());
 
     details1 = updateJobDetails(populateExpected(details1.getUpdate()), 
FIRST_EVENT);
     details2 = updateJobDetails(populateExpected(details2.getUpdate()), 
FIRST_EVENT);
@@ -445,11 +439,7 @@ public abstract class AbstractJobUpdateStoreTest {
     assertEquals(Optional.of(details1), getUpdateDetails(updateId1));
     assertEquals(Optional.of(details2), getUpdateDetails(updateId2));
 
-    assertEquals(
-        ImmutableSet.of(
-            new StoredJobUpdateDetails(details1.newBuilder(), "lock1"),
-            new StoredJobUpdateDetails(details2.newBuilder(), "lock2")),
-        getAllUpdateDetails());
+    assertEquals(ImmutableSet.of(details1, details2), getAllUpdateDetails());
 
     assertEquals(
         ImmutableList.of(getUpdateDetails(updateId2).get(), 
getUpdateDetails(updateId1).get()),
@@ -463,7 +453,7 @@ public abstract class AbstractJobUpdateStoreTest {
     IJobInstanceUpdateEvent instanceEvent = IJobInstanceUpdateEvent.build(
         new JobInstanceUpdateEvent(0, 125L, INSTANCE_ROLLBACK_FAILED));
 
-    saveUpdate(update, Optional.of("lock"));
+    saveUpdate(update);
     saveJobEvent(makeJobUpdateEvent(ROLLING_FORWARD, 123L), updateId);
     saveJobInstanceEvent(instanceEvent, updateId);
     assertEquals(1L, 
stats.getLongValue(jobUpdateActionStatName(INSTANCE_ROLLBACK_FAILED)));
@@ -504,20 +494,13 @@ public abstract class AbstractJobUpdateStoreTest {
     IJobUpdateEvent updateEvent6 = makeJobUpdateEvent(FAILED, 125L);
     IJobUpdateEvent updateEvent7 = makeJobUpdateEvent(ROLLING_FORWARD, 126L);
 
-    update1 = populateExpected(
-        saveUpdateNoEvent(update1, Optional.of("lock1")), ROLLING_BACK, 123L, 
123L);
-    update2 = populateExpected(
-        saveUpdateNoEvent(update2, Optional.absent()), ABORTED, 124L, 124L);
-    update3 = populateExpected(
-        saveUpdateNoEvent(update3, Optional.absent()), ROLLED_BACK, 125L, 
125L);
-    update4 = populateExpected(
-        saveUpdateNoEvent(update4, Optional.absent()), FAILED, 126L, 126L);
-    update5 = populateExpected(
-        saveUpdateNoEvent(update5, Optional.absent()), ERROR, 123L, 123L);
-    update6 = populateExpected(
-        saveUpdateNoEvent(update6, Optional.absent()), FAILED, 125L, 125L);
-    update7 = populateExpected(
-        saveUpdateNoEvent(update7, Optional.of("lock2")), ROLLING_FORWARD, 
126L, 126L);
+    update1 = populateExpected(saveUpdateNoEvent(update1), ROLLING_BACK, 123L, 
123L);
+    update2 = populateExpected(saveUpdateNoEvent(update2), ABORTED, 124L, 
124L);
+    update3 = populateExpected(saveUpdateNoEvent(update3), ROLLED_BACK, 125L, 
125L);
+    update4 = populateExpected(saveUpdateNoEvent(update4), FAILED, 126L, 126L);
+    update5 = populateExpected(saveUpdateNoEvent(update5), ERROR, 123L, 123L);
+    update6 = populateExpected(saveUpdateNoEvent(update6), FAILED, 125L, 125L);
+    update7 = populateExpected(saveUpdateNoEvent(update7), ROLLING_FORWARD, 
126L, 126L);
 
     saveJobEvent(updateEvent1, updateId1);
     saveJobEvent(updateEvent2, updateId2);
@@ -590,16 +573,8 @@ public abstract class AbstractJobUpdateStoreTest {
   @Test(expected = StorageException.class)
   public void testSaveTwoUpdatesForOneJob() {
     IJobUpdate update = makeJobUpdate(makeKey("updateId"));
-    saveUpdate(update, Optional.of("lock1"));
-    saveUpdate(update, Optional.of("lock2"));
-  }
-
-  @Test(expected = StorageException.class)
-  public void testSaveTwoUpdatesSameJobKey() {
-    IJobUpdate update1 = makeJobUpdate(makeKey("update1"));
-    IJobUpdate update2 = makeJobUpdate(makeKey("update2"));
-    saveUpdate(update1, Optional.of("lock1"));
-    saveUpdate(update2, Optional.of("lock1"));
+    saveUpdate(update);
+    saveUpdate(update);
   }
 
   @Test
@@ -614,7 +589,7 @@ public abstract class AbstractJobUpdateStoreTest {
 
     assertEquals(Optional.absent(), getUpdate(updateId));
 
-    saveUpdate(update, Optional.of("lock1"));
+    saveUpdate(update);
     assertUpdate(update);
   }
 
@@ -626,16 +601,11 @@ public abstract class AbstractJobUpdateStoreTest {
     IJobKey job3 = JobKeys.from(role1, "env", "name3");
     IJobKey job4 = JobKeys.from(role1, "env", "name4");
     IJobKey job5 = JobKeys.from("role", "env", "name5");
-    IJobUpdateSummary s1 =
-        saveSummary(makeKey(job1, "u1"), 1230L, ROLLED_BACK, "user", 
Optional.of("lock1"));
-    IJobUpdateSummary s2 =
-        saveSummary(makeKey(job2, "u2"), 1231L, ABORTED, "user", 
Optional.of("lock2"));
-    IJobUpdateSummary s3 =
-        saveSummary(makeKey(job3, "u3"), 1239L, ERROR, "user2", 
Optional.of("lock3"));
-    IJobUpdateSummary s4 =
-        saveSummary(makeKey(job4, "u4"), 1234L, ROLL_BACK_PAUSED, "user3", 
Optional.of("lock4"));
-    IJobUpdateSummary s5 =
-        saveSummary(makeKey(job5, "u5"), 1235L, ROLLING_FORWARD, "user4", 
Optional.of("lock5"));
+    IJobUpdateSummary s1 = saveSummary(makeKey(job1, "u1"), 1230L, 
ROLLED_BACK, "user");
+    IJobUpdateSummary s2 = saveSummary(makeKey(job2, "u2"), 1231L, ABORTED, 
"user");
+    IJobUpdateSummary s3 = saveSummary(makeKey(job3, "u3"), 1239L, ERROR, 
"user2");
+    IJobUpdateSummary s4 = saveSummary(makeKey(job4, "u4"), 1234L, 
ROLL_BACK_PAUSED, "user3");
+    IJobUpdateSummary s5 = saveSummary(makeKey(job5, "u5"), 1235L, 
ROLLING_FORWARD, "user4");
 
     // Test empty query returns all.
     assertEquals(ImmutableList.of(s3, s5, s4, s2, s1), getSummaries(new 
JobUpdateQuery()));
@@ -716,8 +686,8 @@ public abstract class AbstractJobUpdateStoreTest {
 
     assertEquals(ImmutableList.of(), getInstanceEvents(updateId2, 3));
 
-    saveUpdate(update1, Optional.of("lock1"));
-    saveUpdate(update2, Optional.of("lock2"));
+    saveUpdate(update1);
+    saveUpdate(update2);
 
     updateJobDetails(populateExpected(update1), FIRST_EVENT);
     updateJobDetails(populateExpected(update2), FIRST_EVENT);
@@ -776,32 +746,9 @@ public abstract class AbstractJobUpdateStoreTest {
   }
 
   @Test
-  public void testLockAssociation() {
-    IJobKey jobKey = JobKeys.from("role1", "env", "name1");
-    IJobUpdateKey updateId1 = makeKey(jobKey, "u1");
-    IJobUpdateKey updateId2 = makeKey(jobKey, "u2");
-
-    IJobUpdate update1 = makeJobUpdate(updateId1);
-
-    saveUpdate(update1, Optional.of("lock1"));
-    saveJobEvent(makeJobUpdateEvent(ABORTED, 568L), updateId1);
-    storage.write((NoResult.Quiet) storeProvider -> {
-      
storeProvider.getLockStore().removeLock(ILockKey.build(LockKey.job(jobKey.newBuilder())));
-    });
-
-    IJobUpdate update2 = makeJobUpdate(updateId2);
-    saveUpdate(update2, Optional.of("lock2"));
-
-    assertEquals(
-        ImmutableSet.of(Optional.absent(), Optional.of("lock2")),
-        FluentIterable.from(getAllUpdateDetails())
-            .transform(u -> Optional.fromNullable(u.getLockToken())).toSet());
-  }
-
-  @Test
   public void testSaveEventsOutOfChronologicalOrder() {
     IJobUpdate update1 = makeJobUpdate(UPDATE1);
-    saveUpdate(update1, Optional.of("lock1"));
+    saveUpdate(update1);
 
     IJobUpdateEvent event2 = makeJobUpdateEvent(ROLLING_FORWARD, 124);
     IJobUpdateEvent event1 = makeJobUpdateEvent(ROLL_FORWARD_PAUSED, 122);
@@ -853,7 +800,7 @@ public abstract class AbstractJobUpdateStoreTest {
         storeProvider -> 
storeProvider.getJobUpdateStore().fetchJobUpdateDetails(key));
   }
 
-  private Set<StoredJobUpdateDetails> getAllUpdateDetails() {
+  private Set<IJobUpdateDetails> getAllUpdateDetails() {
     return storage.read(
         storeProvider -> 
storeProvider.getJobUpdateStore().fetchAllJobUpdateDetails());
   }
@@ -868,20 +815,9 @@ public abstract class AbstractJobUpdateStoreTest {
         IJobUpdateQuery.build(query)));
   }
 
-  private static ILock makeLock(IJobUpdate update, String lockToken) {
-    return ILock.build(new Lock()
-        
.setKey(LockKey.job(update.getSummary().getKey().getJob().newBuilder()))
-        .setToken(lockToken)
-        .setTimestampMs(100)
-        .setUser("fake user"));
-  }
-
-  private IJobUpdate saveUpdate(IJobUpdate update, Optional<String> lockToken) 
{
+  private IJobUpdate saveUpdate(IJobUpdate update) {
     storage.write((NoResult.Quiet) storeProvider -> {
-      if (lockToken.isPresent()) {
-        storeProvider.getLockStore().saveLock(makeLock(update, 
lockToken.get()));
-      }
-      storeProvider.getJobUpdateStore().saveJobUpdate(update, lockToken);
+      storeProvider.getJobUpdateStore().saveJobUpdate(update);
       storeProvider.getJobUpdateStore().saveJobUpdateEvent(
           update.getSummary().getKey(),
           FIRST_EVENT);
@@ -890,13 +826,9 @@ public abstract class AbstractJobUpdateStoreTest {
     return update;
   }
 
-  private IJobUpdate saveUpdateNoEvent(IJobUpdate update, Optional<String> 
lockToken) {
-    storage.write((NoResult.Quiet) storeProvider -> {
-      if (lockToken.isPresent()) {
-        storeProvider.getLockStore().saveLock(makeLock(update, 
lockToken.get()));
-      }
-      storeProvider.getJobUpdateStore().saveJobUpdate(update, lockToken);
-    });
+  private IJobUpdate saveUpdateNoEvent(IJobUpdate update) {
+    storage.write((NoResult.Quiet) storeProvider ->
+        storeProvider.getJobUpdateStore().saveJobUpdate(update));
 
     return update;
   }
@@ -992,8 +924,7 @@ public abstract class AbstractJobUpdateStoreTest {
       IJobUpdateKey key,
       Long modifiedTimestampMs,
       JobUpdateStatus status,
-      String user,
-      Optional<String> lockToken) {
+      String user) {
 
     IJobUpdateSummary summary = IJobUpdateSummary.build(new JobUpdateSummary()
         .setKey(key.newBuilder())
@@ -1001,7 +932,7 @@ public abstract class AbstractJobUpdateStoreTest {
         .setMetadata(METADATA));
 
     IJobUpdate update = makeJobUpdate(summary);
-    saveUpdate(update, lockToken);
+    saveUpdate(update);
     saveJobEvent(makeJobUpdateEvent(status, modifiedTimestampMs), key);
     return populateExpected(update, status, CREATED_MS, 
modifiedTimestampMs).getSummary();
   }

http://git-wip-us.apache.org/repos/asf/aurora/blob/4fecf1f5/src/test/java/org/apache/aurora/scheduler/storage/AbstractLockStoreTest.java
----------------------------------------------------------------------
diff --git 
a/src/test/java/org/apache/aurora/scheduler/storage/AbstractLockStoreTest.java 
b/src/test/java/org/apache/aurora/scheduler/storage/AbstractLockStoreTest.java
deleted file mode 100644
index b6fd027..0000000
--- 
a/src/test/java/org/apache/aurora/scheduler/storage/AbstractLockStoreTest.java
+++ /dev/null
@@ -1,200 +0,0 @@
-/**
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.aurora.scheduler.storage;
-
-import java.io.IOException;
-import java.util.Optional;
-
-import com.google.common.collect.ImmutableSet;
-
-import org.apache.aurora.gen.JobKey;
-import org.apache.aurora.gen.Lock;
-import org.apache.aurora.gen.LockKey;
-import org.apache.aurora.scheduler.base.JobKeys;
-import org.apache.aurora.scheduler.storage.Storage.StorageException;
-import org.apache.aurora.scheduler.storage.entities.ILock;
-import org.apache.aurora.scheduler.storage.entities.ILockKey;
-import org.apache.aurora.scheduler.storage.testing.StorageEntityUtil;
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-public abstract class AbstractLockStoreTest {
-
-  private Storage storage;
-
-  private void assertLocks(ILock... expected) {
-    assertEquals(
-        ImmutableSet.<ILock>builder().add(expected).build(),
-        storage.read(storeProvider -> 
storeProvider.getLockStore().fetchLocks()));
-  }
-
-  private Optional<ILock> getLock(ILockKey key) {
-    return storage.read(storeProvider -> 
storeProvider.getLockStore().fetchLock(key));
-  }
-
-  private void saveLocks(ILock... locks) {
-    storage.write(storeProvider -> {
-      for (ILock lock : locks) {
-        storeProvider.getLockStore().saveLock(lock);
-      }
-      return null;
-    });
-  }
-
-  private void removeLocks(ILock... locks) {
-    storage.write(storeProvider -> {
-      for (ILock lock : locks) {
-        storeProvider.getLockStore().removeLock(lock.getKey());
-      }
-      return null;
-    });
-  }
-
-  private static ILock makeLock(JobKey key, String token) {
-    return ILock.build(new Lock()
-      .setKey(LockKey.job(key))
-      .setToken(token)
-      .setUser("testUser")
-      .setMessage("Test message")
-      .setTimestampMs(12345L));
-  }
-
-  @Before
-  public void setUp() throws IOException {
-    storage = createStorage();
-  }
-
-  protected abstract Storage createStorage();
-
-  @Test
-  public void testLocks() throws Exception {
-    assertLocks();
-
-    String role = "testRole";
-    String env = "testEnv";
-    String job1 = "testJob1";
-    String job2 = "testJob2";
-
-    ILock lock1 = makeLock(JobKeys.from(role, env, job1).newBuilder(), 
"token1");
-    ILock lock2 = makeLock(JobKeys.from(role, env, job2).newBuilder(), 
"token2");
-
-    saveLocks(lock1, lock2);
-    assertLocks(lock1, lock2);
-    removeLocks(lock1);
-
-    assertLocks(lock2);
-  }
-
-  @Test
-  public void testRepeatedWrite() throws Exception {
-    assertLocks();
-
-    String role = "testRole";
-    String env = "testEnv";
-    String job = "testJob";
-
-    ILock lock = makeLock(JobKeys.from(role, env, job).newBuilder(), "token1");
-
-    saveLocks(lock);
-    try {
-      saveLocks(lock);
-      fail("saveLock should have failed unique constraint check.");
-    } catch (StorageException e) {
-      // expected
-    }
-
-    assertLocks(lock);
-  }
-
-  @Test
-  public void testExistingJobKey() throws Exception {
-    String role = "testRole";
-    String env = "testEnv";
-    String job = "testJob";
-
-    ILock lock = makeLock(JobKeys.from(role, env, job).newBuilder(), "token1");
-
-    saveLocks(lock);
-    removeLocks(lock);
-    saveLocks(lock);
-
-    assertLocks(lock);
-  }
-
-  @Test
-  public void testGetLock() throws Exception {
-    assertLocks();
-
-    String role1 = "testRole1";
-    String role2 = "testRole2";
-    String env = "testEnv";
-    String job = "testJob";
-
-    ILock lock1 = makeLock(JobKeys.from(role1, env, job).newBuilder(), 
"token1");
-    ILock lock2 = makeLock(JobKeys.from(role2, env, job).newBuilder(), 
"token2");
-
-    assertEquals(Optional.empty(), getLock(lock1.getKey()));
-    assertEquals(Optional.empty(), getLock(lock2.getKey()));
-
-    saveLocks(StorageEntityUtil.assertFullyPopulated(lock1));
-    assertEquals(Optional.of(lock1), getLock(lock1.getKey()));
-    assertEquals(Optional.empty(), getLock(lock2.getKey()));
-    saveLocks(lock2);
-    assertEquals(Optional.of(lock1), getLock(lock1.getKey()));
-    assertEquals(Optional.of(lock2), getLock(lock2.getKey()));
-    removeLocks(lock1);
-    assertEquals(Optional.empty(), getLock(lock1.getKey()));
-    assertEquals(Optional.of(lock2), getLock(lock2.getKey()));
-  }
-
-  @Test
-  public void testDeleteAllLocks() throws Exception {
-    assertLocks();
-
-    String role = "testRole";
-    String env = "testEnv";
-    String job1 = "testJob1";
-    String job2 = "testJob2";
-
-    ILock lock1 = makeLock(JobKeys.from(role, env, job1).newBuilder(), 
"token1");
-    ILock lock2 = makeLock(JobKeys.from(role, env, job2).newBuilder(), 
"token2");
-
-    saveLocks(lock1, lock2);
-    assertLocks(lock1, lock2);
-
-    storage.write(storeProvider -> {
-      storeProvider.getLockStore().deleteLocks();
-      return null;
-    });
-
-    assertLocks();
-  }
-
-  @Test
-  public void testDuplicateToken() throws Exception {
-    ILock lock = makeLock(JobKeys.from("role", "env", "job1").newBuilder(), 
"token1");
-    saveLocks(lock);
-    try {
-      saveLocks(makeLock(JobKeys.from("role", "env", "job2").newBuilder(), 
"token1"));
-      fail();
-    } catch (StorageException e) {
-      // Expected.
-    }
-
-    assertLocks(lock);
-  }
-}

http://git-wip-us.apache.org/repos/asf/aurora/blob/4fecf1f5/src/test/java/org/apache/aurora/scheduler/storage/backup/RecoveryTest.java
----------------------------------------------------------------------
diff --git 
a/src/test/java/org/apache/aurora/scheduler/storage/backup/RecoveryTest.java 
b/src/test/java/org/apache/aurora/scheduler/storage/backup/RecoveryTest.java
index bcc7438..7138d6b 100644
--- a/src/test/java/org/apache/aurora/scheduler/storage/backup/RecoveryTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/storage/backup/RecoveryTest.java
@@ -171,7 +171,6 @@ public class RecoveryTest extends EasyMockTest {
     return new Snapshot()
         .setHostAttributes(ImmutableSet.of())
         .setQuotaConfigurations(ImmutableSet.of())
-        .setLocks(ImmutableSet.of())
         .setJobUpdateDetails(ImmutableSet.of())
         .setCronJobs(ImmutableSet.of())
         .setSchedulerMetadata(metadata)

http://git-wip-us.apache.org/repos/asf/aurora/blob/4fecf1f5/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 17e75c5..c208210 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
@@ -48,8 +48,6 @@ import org.apache.aurora.gen.JobUpdateKey;
 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.MaintenanceMode;
 import org.apache.aurora.gen.Range;
 import org.apache.aurora.gen.ResourceAggregate;
@@ -61,7 +59,6 @@ import org.apache.aurora.gen.storage.LogEntry;
 import org.apache.aurora.gen.storage.Op;
 import org.apache.aurora.gen.storage.PruneJobUpdateHistory;
 import org.apache.aurora.gen.storage.RemoveJob;
-import org.apache.aurora.gen.storage.RemoveLock;
 import org.apache.aurora.gen.storage.RemoveQuota;
 import org.apache.aurora.gen.storage.RemoveTasks;
 import org.apache.aurora.gen.storage.SaveCronJob;
@@ -70,7 +67,6 @@ import org.apache.aurora.gen.storage.SaveHostAttributes;
 import org.apache.aurora.gen.storage.SaveJobInstanceUpdateEvent;
 import org.apache.aurora.gen.storage.SaveJobUpdate;
 import org.apache.aurora.gen.storage.SaveJobUpdateEvent;
-import org.apache.aurora.gen.storage.SaveLock;
 import org.apache.aurora.gen.storage.SaveQuota;
 import org.apache.aurora.gen.storage.SaveTasks;
 import org.apache.aurora.gen.storage.Snapshot;
@@ -99,8 +95,6 @@ import org.apache.aurora.scheduler.storage.entities.IJobKey;
 import org.apache.aurora.scheduler.storage.entities.IJobUpdate;
 import org.apache.aurora.scheduler.storage.entities.IJobUpdateEvent;
 import org.apache.aurora.scheduler.storage.entities.IJobUpdateKey;
-import org.apache.aurora.scheduler.storage.entities.ILock;
-import org.apache.aurora.scheduler.storage.entities.ILockKey;
 import org.apache.aurora.scheduler.storage.entities.IResourceAggregate;
 import org.apache.aurora.scheduler.storage.entities.IScheduledTask;
 import org.apache.aurora.scheduler.storage.log.LogStorage.SchedulingService;
@@ -174,7 +168,6 @@ public class LogStorageTest extends EasyMockTest {
         storageUtil.schedulerStore,
         storageUtil.jobStore,
         storageUtil.taskStore,
-        storageUtil.lockStore,
         storageUtil.quotaStore,
         storageUtil.attributeStore,
         storageUtil.jobUpdateStore,
@@ -327,14 +320,6 @@ public class LogStorageTest extends EasyMockTest {
     expect(storageUtil.attributeStore.saveHostAttributes(
         
IHostAttributes.build(hostAttributes2.getHostAttributes()))).andReturn(true);
 
-    SaveLock saveLock = new SaveLock(new 
Lock().setKey(LockKey.job(JOB_KEY.newBuilder())));
-    builder.add(createTransaction(Op.saveLock(saveLock)));
-    storageUtil.lockStore.saveLock(ILock.build(saveLock.getLock()));
-
-    RemoveLock removeLock = new RemoveLock(LockKey.job(JOB_KEY.newBuilder()));
-    builder.add(createTransaction(Op.removeLock(removeLock)));
-    storageUtil.lockStore.removeLock(ILockKey.build(removeLock.getLockKey()));
-
     JobUpdate actualUpdate = new JobUpdate()
         .setSummary(new JobUpdateSummary().setKey(UPDATE_ID.newBuilder()))
         .setInstructions(new JobUpdateInstructions()
@@ -345,11 +330,9 @@ public class LogStorageTest extends EasyMockTest {
     
expectedUpdate.getInstructions().getDesiredState().setTask(makeConfig(JOB_KEY).newBuilder());
     expectedUpdate.getInstructions().getInitialState()
         .forEach(e -> e.setTask(makeConfig(JOB_KEY).newBuilder()));
-    SaveJobUpdate saveUpdate = new SaveJobUpdate(actualUpdate, "token");
+    SaveJobUpdate saveUpdate = new SaveJobUpdate().setJobUpdate(actualUpdate);
     builder.add(createTransaction(Op.saveJobUpdate(saveUpdate)));
-    storageUtil.jobUpdateStore.saveJobUpdate(
-        IJobUpdate.build(expectedUpdate),
-        Optional.of(saveUpdate.getLockToken()));
+    storageUtil.jobUpdateStore.saveJobUpdate(IJobUpdate.build(expectedUpdate));
 
     SaveJobUpdateEvent saveUpdateEvent =
         new SaveJobUpdateEvent(new JobUpdateEvent(), UPDATE_ID.newBuilder());
@@ -745,48 +728,6 @@ public class LogStorageTest extends EasyMockTest {
   }
 
   @Test
-  public void testSaveLock() throws Exception {
-    ILock lock = ILock.build(new Lock()
-        .setKey(LockKey.job(JOB_KEY.newBuilder()))
-        .setToken("testLockId")
-        .setUser("testUser")
-        .setTimestampMs(12345L));
-    new AbstractMutationFixture() {
-      @Override
-      protected void setupExpectations() throws Exception {
-        storageUtil.expectWrite();
-        storageUtil.lockStore.saveLock(lock);
-        streamMatcher.expectTransaction(Op.saveLock(new 
SaveLock(lock.newBuilder())))
-            .andReturn(position);
-      }
-
-      @Override
-      protected void performMutations(MutableStoreProvider storeProvider) {
-        storeProvider.getLockStore().saveLock(lock);
-      }
-    }.run();
-  }
-
-  @Test
-  public void testRemoveLock() throws Exception {
-    ILockKey lockKey = ILockKey.build(LockKey.job(JOB_KEY.newBuilder()));
-    new AbstractMutationFixture() {
-      @Override
-      protected void setupExpectations() throws Exception {
-        storageUtil.expectWrite();
-        storageUtil.lockStore.removeLock(lockKey);
-        streamMatcher.expectTransaction(Op.removeLock(new 
RemoveLock(lockKey.newBuilder())))
-            .andReturn(position);
-      }
-
-      @Override
-      protected void performMutations(MutableStoreProvider storeProvider) {
-        storeProvider.getLockStore().removeLock(lockKey);
-      }
-    }.run();
-  }
-
-  @Test
   public void testSaveHostAttributes() throws Exception {
     String host = "hostname";
     Set<Attribute> attributes =
@@ -834,16 +775,7 @@ public class LogStorageTest extends EasyMockTest {
   }
 
   @Test
-  public void testSaveUpdateWithLockToken() throws Exception {
-    saveAndAssertJobUpdate(Optional.of("token"));
-  }
-
-  @Test
-  public void testSaveUpdateWithNullLockToken() throws Exception {
-    saveAndAssertJobUpdate(Optional.absent());
-  }
-
-  private void saveAndAssertJobUpdate(Optional<String> lockToken) throws 
Exception {
+  public void testSaveUpdate() throws Exception {
     IJobUpdate update = IJobUpdate.build(new JobUpdate()
         .setSummary(new JobUpdateSummary()
             .setKey(UPDATE_ID.newBuilder())
@@ -861,15 +793,15 @@ public class LogStorageTest extends EasyMockTest {
       @Override
       protected void setupExpectations() throws Exception {
         storageUtil.expectWrite();
-        storageUtil.jobUpdateStore.saveJobUpdate(update, lockToken);
+        storageUtil.jobUpdateStore.saveJobUpdate(update);
         streamMatcher.expectTransaction(
-            Op.saveJobUpdate(new SaveJobUpdate(update.newBuilder(), 
lockToken.orNull())))
+            Op.saveJobUpdate(new 
SaveJobUpdate().setJobUpdate(update.newBuilder())))
             .andReturn(position);
       }
 
       @Override
       protected void performMutations(MutableStoreProvider storeProvider) {
-        storeProvider.getJobUpdateStore().saveJobUpdate(update, lockToken);
+        storeProvider.getJobUpdateStore().saveJobUpdate(update);
       }
     }.run();
   }

http://git-wip-us.apache.org/repos/asf/aurora/blob/4fecf1f5/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplIT.java
----------------------------------------------------------------------
diff --git 
a/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplIT.java
 
b/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplIT.java
index 2d161c8..a1944c4 100644
--- 
a/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplIT.java
+++ 
b/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplIT.java
@@ -44,8 +44,6 @@ import org.apache.aurora.gen.JobUpdateSettings;
 import org.apache.aurora.gen.JobUpdateState;
 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.MaintenanceMode;
 import org.apache.aurora.gen.Range;
 import org.apache.aurora.gen.storage.QuotaConfiguration;
@@ -62,7 +60,6 @@ import 
org.apache.aurora.scheduler.storage.entities.IJobConfiguration;
 import org.apache.aurora.scheduler.storage.entities.IJobKey;
 import org.apache.aurora.scheduler.storage.entities.IJobUpdateDetails;
 import org.apache.aurora.scheduler.storage.entities.IJobUpdateKey;
-import org.apache.aurora.scheduler.storage.entities.ILock;
 import org.apache.aurora.scheduler.storage.entities.IResourceAggregate;
 import org.apache.aurora.scheduler.storage.entities.IScheduledTask;
 import org.apache.aurora.scheduler.storage.entities.ITaskConfig;
@@ -136,11 +133,6 @@ public class SnapshotStoreImplIT {
           FakeBuildInfo.DATE, FakeBuildInfo.DATE,
           FakeBuildInfo.GIT_REVISION, FakeBuildInfo.GIT_REVISION,
           FakeBuildInfo.GIT_TAG, FakeBuildInfo.GIT_TAG);
-  private static final ILock LOCK = ILock.build(new Lock()
-      .setKey(LockKey.job(JobKeys.from("role", "env", "job").newBuilder()))
-      .setToken("lockId")
-      .setUser("testUser")
-      .setTimestampMs(12345L));
   private static final IJobUpdateKey UPDATE_ID =
       IJobUpdateKey.build(new JobUpdateKey(JOB_KEY.newBuilder(), "updateId1"));
   private static final IJobUpdateDetails UPDATE = IJobUpdateDetails.build(new 
JobUpdateDetails()
@@ -181,9 +173,8 @@ public class SnapshotStoreImplIT {
         .setHostAttributes(ImmutableSet.of(ATTRIBUTES.newBuilder()))
         .setCronJobs(ImmutableSet.of(new StoredCronJob(CRON_JOB.newBuilder())))
         .setSchedulerMetadata(new SchedulerMetadata(FRAMEWORK_ID, METADATA))
-        .setLocks(ImmutableSet.of(LOCK.newBuilder()))
         .setJobUpdateDetails(ImmutableSet.of(
-            new StoredJobUpdateDetails(UPDATE.newBuilder(), LOCK.getToken())));
+            new StoredJobUpdateDetails().setDetails(UPDATE.newBuilder())));
   }
 
   private Snapshot makeNonBackfilled() {

http://git-wip-us.apache.org/repos/asf/aurora/blob/4fecf1f5/src/test/java/org/apache/aurora/scheduler/storage/log/WriteAheadStorageTest.java
----------------------------------------------------------------------
diff --git 
a/src/test/java/org/apache/aurora/scheduler/storage/log/WriteAheadStorageTest.java
 
b/src/test/java/org/apache/aurora/scheduler/storage/log/WriteAheadStorageTest.java
index 13726cc..d5e5c11 100644
--- 
a/src/test/java/org/apache/aurora/scheduler/storage/log/WriteAheadStorageTest.java
+++ 
b/src/test/java/org/apache/aurora/scheduler/storage/log/WriteAheadStorageTest.java
@@ -35,7 +35,6 @@ import org.apache.aurora.scheduler.events.PubsubEvent;
 import org.apache.aurora.scheduler.storage.AttributeStore;
 import org.apache.aurora.scheduler.storage.CronJobStore;
 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;
 import org.apache.aurora.scheduler.storage.TaskStore;
@@ -73,7 +72,6 @@ public class WriteAheadStorageTest extends EasyMockTest {
         createMock(SchedulerStore.Mutable.class),
         createMock(CronJobStore.Mutable.class),
         taskStore,
-        createMock(LockStore.Mutable.class),
         createMock(QuotaStore.Mutable.class),
         attributeStore,
         jobUpdateStore,
@@ -172,12 +170,6 @@ public class WriteAheadStorageTest extends EasyMockTest {
   }
 
   @Test(expected = UnsupportedOperationException.class)
-  public void testDeleteLocks() {
-    control.replay();
-    storage.deleteLocks();
-  }
-
-  @Test(expected = UnsupportedOperationException.class)
   public void testDeleteAllUpdatesAndEvents() {
     control.replay();
     storage.deleteAllUpdatesAndEvents();

http://git-wip-us.apache.org/repos/asf/aurora/blob/4fecf1f5/src/test/java/org/apache/aurora/scheduler/storage/mem/MemLockStoreTest.java
----------------------------------------------------------------------
diff --git 
a/src/test/java/org/apache/aurora/scheduler/storage/mem/MemLockStoreTest.java 
b/src/test/java/org/apache/aurora/scheduler/storage/mem/MemLockStoreTest.java
deleted file mode 100644
index 4019d1a..0000000
--- 
a/src/test/java/org/apache/aurora/scheduler/storage/mem/MemLockStoreTest.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/**
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.aurora.scheduler.storage.mem;
-
-import org.apache.aurora.scheduler.storage.AbstractLockStoreTest;
-import org.apache.aurora.scheduler.storage.Storage;
-
-public class MemLockStoreTest extends AbstractLockStoreTest {
-  @Override
-  protected Storage createStorage() {
-    return MemStorageModule.newEmptyStorage();
-  }
-}

http://git-wip-us.apache.org/repos/asf/aurora/blob/4fecf1f5/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 21d26b3..fd81bff 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
@@ -21,7 +21,6 @@ import org.apache.aurora.scheduler.base.Query;
 import org.apache.aurora.scheduler.storage.AttributeStore;
 import org.apache.aurora.scheduler.storage.CronJobStore;
 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;
 import org.apache.aurora.scheduler.storage.Storage.MutableStoreProvider;
@@ -50,7 +49,6 @@ public class StorageTestUtil {
   public final QuotaStore.Mutable quotaStore;
   public final AttributeStore.Mutable attributeStore;
   public final CronJobStore.Mutable jobStore;
-  public final LockStore.Mutable lockStore;
   public final SchedulerStore.Mutable schedulerStore;
   public final JobUpdateStore.Mutable jobUpdateStore;
   public final NonVolatileStorage storage;
@@ -67,7 +65,6 @@ public class StorageTestUtil {
     this.quotaStore = easyMock.createMock(QuotaStore.Mutable.class);
     this.attributeStore = easyMock.createMock(AttributeStore.Mutable.class);
     this.jobStore = easyMock.createMock(CronJobStore.Mutable.class);
-    this.lockStore = easyMock.createMock(LockStore.Mutable.class);
     this.schedulerStore = easyMock.createMock(SchedulerStore.Mutable.class);
     this.jobUpdateStore = easyMock.createMock(JobUpdateStore.Mutable.class);
     this.storage = easyMock.createMock(NonVolatileStorage.class);
@@ -93,7 +90,6 @@ public class StorageTestUtil {
     expect(storeProvider.getQuotaStore()).andReturn(quotaStore).anyTimes();
     
expect(storeProvider.getAttributeStore()).andReturn(attributeStore).anyTimes();
     expect(storeProvider.getCronJobStore()).andReturn(jobStore).anyTimes();
-    expect(storeProvider.getLockStore()).andReturn(lockStore).anyTimes();
     
expect(storeProvider.getSchedulerStore()).andReturn(schedulerStore).anyTimes();
     
expect(storeProvider.getJobUpdateStore()).andReturn(jobUpdateStore).anyTimes();
     
expect(mutableStoreProvider.getTaskStore()).andReturn(taskStore).anyTimes();
@@ -101,7 +97,6 @@ public class StorageTestUtil {
     
expect(mutableStoreProvider.getQuotaStore()).andReturn(quotaStore).anyTimes();
     
expect(mutableStoreProvider.getAttributeStore()).andReturn(attributeStore).anyTimes();
     
expect(mutableStoreProvider.getCronJobStore()).andReturn(jobStore).anyTimes();
-    
expect(mutableStoreProvider.getLockStore()).andReturn(lockStore).anyTimes();
     
expect(mutableStoreProvider.getSchedulerStore()).andReturn(schedulerStore).anyTimes();
     
expect(mutableStoreProvider.getJobUpdateStore()).andReturn(jobUpdateStore).anyTimes();
     expectRead().anyTimes();

http://git-wip-us.apache.org/repos/asf/aurora/blob/4fecf1f5/src/test/java/org/apache/aurora/scheduler/thrift/Fixtures.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/thrift/Fixtures.java 
b/src/test/java/org/apache/aurora/scheduler/thrift/Fixtures.java
index 43e32ee..d1b5e40 100644
--- a/src/test/java/org/apache/aurora/scheduler/thrift/Fixtures.java
+++ b/src/test/java/org/apache/aurora/scheduler/thrift/Fixtures.java
@@ -35,7 +35,6 @@ import org.apache.aurora.gen.JobConfiguration;
 import org.apache.aurora.gen.JobSummary;
 import org.apache.aurora.gen.JobSummaryResult;
 import org.apache.aurora.gen.JobUpdateKey;
-import org.apache.aurora.gen.LockKey;
 import org.apache.aurora.gen.MesosContainer;
 import org.apache.aurora.gen.Resource;
 import org.apache.aurora.gen.Response;
@@ -52,7 +51,6 @@ import org.apache.aurora.scheduler.resources.ResourceBag;
 import org.apache.aurora.scheduler.resources.ResourceTestUtil;
 import org.apache.aurora.scheduler.storage.entities.IJobKey;
 import org.apache.aurora.scheduler.storage.entities.IJobUpdateKey;
-import org.apache.aurora.scheduler.storage.entities.ILockKey;
 import org.apache.aurora.scheduler.storage.entities.IResult;
 import org.apache.aurora.scheduler.storage.entities.IScheduledTask;
 
@@ -67,7 +65,6 @@ final class Fixtures {
   static final Identity IDENTITY = new Identity().setUser(USER);
   static final String JOB_NAME = "job_foo";
   static final IJobKey JOB_KEY = JobKeys.from(ROLE, "devel", JOB_NAME);
-  static final ILockKey LOCK_KEY = 
ILockKey.build(LockKey.job(JOB_KEY.newBuilder()));
   static final JobConfiguration CRON_JOB = makeJob().setCronSchedule("* * * * 
*");
   static final String TASK_ID = "task_id";
   static final String UPDATE_ID = "82d6d790-3212-11e3-aa6e-0800200c9a74";

http://git-wip-us.apache.org/repos/asf/aurora/blob/4fecf1f5/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
----------------------------------------------------------------------
diff --git 
a/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
 
b/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
index 1691477..42a79a6 100644
--- 
a/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
+++ 
b/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
@@ -253,7 +253,7 @@ public class SchedulerThriftInterfaceTest extends 
EasyMockTest {
   }
 
   @Test
-  public void testCreateJobNoLock() throws Exception {
+  public void testCreateJob() throws Exception {
     IJobConfiguration job = IJobConfiguration.build(makeProdJob());
     SanitizedConfiguration sanitized = fromUnsanitized(job);
     jobUpdateController.assertNotUpdating(JOB_KEY);
@@ -273,26 +273,6 @@ public class SchedulerThriftInterfaceTest extends 
EasyMockTest {
   }
 
   @Test
-  public void testCreateJobWithLock() throws Exception {
-    IJobConfiguration job = IJobConfiguration.build(makeProdJob());
-    SanitizedConfiguration sanitized = fromUnsanitized(job);
-    jobUpdateController.assertNotUpdating(JOB_KEY);
-    storageUtil.expectTaskFetch(Query.jobScoped(JOB_KEY).active());
-    expectNoCronJob();
-    expectInstanceQuotaCheck(sanitized, ENOUGH_QUOTA);
-
-    stateManager.insertPendingTasks(
-        storageUtil.mutableStoreProvider,
-        sanitized.getJobConfig().getTaskConfig(),
-        sanitized.getInstanceIds());
-
-    control.replay();
-
-    assertOkResponse(thrift.createJob(job.newBuilder()));
-    assertEquals(1L, statsProvider.getLongValue(CREATE_JOB));
-  }
-
-  @Test
   public void testCreateJobFailsForCron() throws Exception {
     IJobConfiguration job = 
IJobConfiguration.build(makeProdJob().setCronSchedule(""));
 
@@ -312,18 +292,6 @@ public class SchedulerThriftInterfaceTest extends 
EasyMockTest {
   }
 
   @Test
-  public void testCreateJobFailsLockCheck() throws Exception {
-    IJobConfiguration job = IJobConfiguration.build(makeJob());
-    jobUpdateController.assertNotUpdating(JOB_KEY);
-    expectLastCall().andThrow(new JobUpdatingException("Job is updating"));
-
-    control.replay();
-
-    assertResponse(JOB_UPDATING_ERROR, thrift.createJob(job.newBuilder()));
-    assertEquals(0L, statsProvider.getLongValue(CREATE_JOB));
-  }
-
-  @Test
   public void testCreateJobFailsJobExists() throws Exception {
     IJobConfiguration job = IJobConfiguration.build(makeJob());
     jobUpdateController.assertNotUpdating(JOB_KEY);
@@ -636,7 +604,7 @@ public class SchedulerThriftInterfaceTest extends 
EasyMockTest {
   }
 
   @Test
-  public void testInstanceScoped() throws Exception {
+  public void testKillTasksInstanceScoped() throws Exception {
     Query.Builder query = Query.instanceScoped(JOB_KEY, 
ImmutableSet.of(1)).active();
     storageUtil.expectTaskFetch(query, buildScheduledTask());
     jobUpdateController.assertNotUpdating(JOB_KEY);
@@ -649,19 +617,13 @@ public class SchedulerThriftInterfaceTest extends 
EasyMockTest {
   }
 
   @Test
-  public void testKillTasksLockCheckFailed() throws Exception {
-    Query.Builder query = Query.unscoped().byJob(JOB_KEY).active();
-    IScheduledTask task2 = buildScheduledTask("job_bar", TASK_ID);
-    storageUtil.expectTaskFetch(query, buildScheduledTask(), task2);
+  public void testKillTasksWhileJobUpdating() throws Exception {
     jobUpdateController.assertNotUpdating(JOB_KEY);
-    
jobUpdateController.assertNotUpdating(task2.getAssignedTask().getTask().getJob());
-    expectLastCall().andThrow(new JobUpdatingException("Job is updating"));
+    expectLastCall().andThrow(new JobUpdatingException("job is updating"));
 
     control.replay();
 
-    assertResponse(
-        JOB_UPDATING_ERROR,
-        thrift.killTasks(JOB_KEY.newBuilder(), null, null));
+    assertResponse(JOB_UPDATING_ERROR, thrift.killTasks(JOB_KEY.newBuilder(), 
null, null));
     assertEquals(0L, statsProvider.getLongValue(KILL_TASKS));
   }
 
@@ -669,6 +631,7 @@ public class SchedulerThriftInterfaceTest extends 
EasyMockTest {
   public void testKillNonExistentTasks() throws Exception {
     Query.Builder query = Query.unscoped().byJob(JOB_KEY).active();
     storageUtil.expectTaskFetch(query);
+    jobUpdateController.assertNotUpdating(JOB_KEY);
 
     control.replay();
 
@@ -891,7 +854,7 @@ public class SchedulerThriftInterfaceTest extends 
EasyMockTest {
   }
 
   @Test
-  public void testRestartShardsLockCheckFails() throws Exception {
+  public void testRestartShardsWhileJobUpdating() throws Exception {
     Set<Integer> shards = ImmutableSet.of(1, 6);
 
     jobUpdateController.assertNotUpdating(JOB_KEY);
@@ -934,16 +897,6 @@ public class SchedulerThriftInterfaceTest extends 
EasyMockTest {
   }
 
   @Test
-  public void testReplaceCronTemplateFailedLockValidation() throws Exception {
-    jobUpdateController.assertNotUpdating(JOB_KEY);
-    expectLastCall().andThrow(new JobUpdatingException("job is updating"));
-    control.replay();
-
-    assertResponse(JOB_UPDATING_ERROR, thrift.replaceCronTemplate(CRON_JOB));
-    assertEquals(0L, statsProvider.getLongValue(CREATE_OR_UPDATE_CRON));
-  }
-
-  @Test
   public void testReplaceCronTemplateDoesNotExist() throws Exception {
     jobUpdateController.assertNotUpdating(JOB_KEY);
     SanitizedConfiguration sanitized = 
fromUnsanitized(IJobConfiguration.build(CRON_JOB));
@@ -1024,14 +977,6 @@ public class SchedulerThriftInterfaceTest extends 
EasyMockTest {
   }
 
   @Test
-  public void testScheduleCronJobFailsLockValidation() throws Exception {
-    jobUpdateController.assertNotUpdating(JOB_KEY);
-    expectLastCall().andThrow(new JobUpdatingException("job is updating"));
-    control.replay();
-    assertResponse(JOB_UPDATING_ERROR, thrift.scheduleCronJob(CRON_JOB));
-  }
-
-  @Test
   public void testScheduleCronJobFailsWithNoCronSchedule() throws Exception {
     control.replay();
 
@@ -1062,7 +1007,7 @@ public class SchedulerThriftInterfaceTest extends 
EasyMockTest {
   }
 
   @Test
-  public void testDescheduleCronJobFailsLockValidation() throws Exception {
+  public void testDescheduleCronJobWhileJobUpdating() throws Exception {
     jobUpdateController.assertNotUpdating(JOB_KEY);
     expectLastCall().andThrow(new JobUpdatingException("job is updating"));
     control.replay();
@@ -1268,10 +1213,6 @@ public class SchedulerThriftInterfaceTest extends 
EasyMockTest {
 
   @Test
   public void testAddInstancesWithInstanceKeyFailsInvalidCount() throws 
Exception {
-    expectNoCronJob();
-    jobUpdateController.assertNotUpdating(JOB_KEY);
-    storageUtil.expectTaskFetch(Query.jobScoped(JOB_KEY).active());
-
     control.replay();
 
     assertEquals(
@@ -1299,7 +1240,7 @@ public class SchedulerThriftInterfaceTest extends 
EasyMockTest {
   }
 
   @Test
-  public void testAddInstancesLockCheckFails() throws Exception {
+  public void testAddInstancesWhileJobUpdating() throws Exception {
     expectNoCronJob();
     jobUpdateController.assertNotUpdating(JOB_KEY);
     expectLastCall().andThrow(new JobUpdatingException("job is updating"));

http://git-wip-us.apache.org/repos/asf/aurora/blob/4fecf1f5/src/test/java/org/apache/aurora/scheduler/updater/JobUpdaterIT.java
----------------------------------------------------------------------
diff --git 
a/src/test/java/org/apache/aurora/scheduler/updater/JobUpdaterIT.java 
b/src/test/java/org/apache/aurora/scheduler/updater/JobUpdaterIT.java
index 4d62bba..661ce58 100644
--- a/src/test/java/org/apache/aurora/scheduler/updater/JobUpdaterIT.java
+++ b/src/test/java/org/apache/aurora/scheduler/updater/JobUpdaterIT.java
@@ -19,7 +19,6 @@ import java.util.concurrent.ScheduledExecutorService;
 
 import com.google.common.base.Function;
 import com.google.common.base.Optional;
-import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableMultimap;
@@ -73,8 +72,6 @@ import org.apache.aurora.scheduler.events.PubsubEvent;
 import org.apache.aurora.scheduler.mesos.Driver;
 import org.apache.aurora.scheduler.scheduling.RescheduleCalculator;
 import 
org.apache.aurora.scheduler.scheduling.RescheduleCalculator.RescheduleCalculatorImpl;
-import org.apache.aurora.scheduler.state.LockManager;
-import org.apache.aurora.scheduler.state.LockManagerImpl;
 import org.apache.aurora.scheduler.state.StateChangeResult;
 import org.apache.aurora.scheduler.state.StateManager;
 import org.apache.aurora.scheduler.state.StateManagerImpl;
@@ -90,7 +87,6 @@ import 
org.apache.aurora.scheduler.storage.entities.IJobUpdateDetails;
 import org.apache.aurora.scheduler.storage.entities.IJobUpdateEvent;
 import org.apache.aurora.scheduler.storage.entities.IJobUpdateKey;
 import org.apache.aurora.scheduler.storage.entities.IJobUpdateSummary;
-import org.apache.aurora.scheduler.storage.entities.ILock;
 import org.apache.aurora.scheduler.storage.entities.IScheduledTask;
 import org.apache.aurora.scheduler.storage.entities.ITaskConfig;
 import org.apache.aurora.scheduler.storage.mem.MemStorageModule;
@@ -156,7 +152,6 @@ public class JobUpdaterIT extends EasyMockTest {
   private Driver driver;
   private EventBus eventBus;
   private Storage storage;
-  private LockManager lockManager;
   private StateManager stateManager;
   private JobUpdateEventSubscriber subscriber;
   private Command shutdownCommand;
@@ -200,7 +195,6 @@ public class JobUpdaterIT extends EasyMockTest {
                     FLAPPING_THRESHOLD,
                     new TimeAmount(1, Time.MINUTES)));
             bind(EventSink.class).toInstance(eventBus::post);
-            bind(LockManager.class).to(LockManagerImpl.class);
             bind(UUIDGenerator.class).to(UUIDGeneratorImpl.class);
             bind(Lifecycle.class).toInstance(new Lifecycle(shutdownCommand));
             bind(TaskEventBatchWorker.class).toInstance(batchWorker);
@@ -209,7 +203,6 @@ public class JobUpdaterIT extends EasyMockTest {
     updater = injector.getInstance(JobUpdateController.class);
     storage = injector.getInstance(Storage.class);
     storage.prepare();
-    lockManager = injector.getInstance(LockManager.class);
     stateManager = injector.getInstance(StateManager.class);
     eventBus.register(injector.getInstance(JobUpdateEventSubscriber.class));
     subscriber = injector.getInstance(JobUpdateEventSubscriber.class);
@@ -219,19 +212,6 @@ public class JobUpdaterIT extends EasyMockTest {
   @After
   public void validateExitState() {
     clock.assertEmpty();
-    assertEquals(ImmutableSet.of(), storage.read(p -> 
p.getLockStore().fetchLocks()));
-  }
-
-  @Test(expected = UpdateStateException.class)
-  public void testJobLocked() throws Exception {
-    control.replay();
-
-    lockManager.acquireLock(JOB, USER);
-    try {
-      updater.start(makeJobUpdate(makeInstanceConfig(0, 0, NEW_CONFIG)), 
AUDIT);
-    } finally {
-      lockManager.releaseLock(JOB);
-    }
   }
 
   private String getTaskId(IJobKey job, int instanceId) {
@@ -263,12 +243,7 @@ public class JobUpdaterIT extends EasyMockTest {
   }
 
   private static final Ordering<IJobInstanceUpdateEvent> EVENT_ORDER = 
Ordering.natural()
-      .onResultOf(new Function<IJobInstanceUpdateEvent, Long>() {
-        @Override
-        public Long apply(IJobInstanceUpdateEvent event) {
-          return event.getTimestampMs();
-        }
-      });
+      .onResultOf(IJobInstanceUpdateEvent::getTimestampMs);
   private static final Function<IJobInstanceUpdateEvent, Integer> 
EVENT_TO_INSTANCE =
       IJobInstanceUpdateEvent::getInstanceId;
 
@@ -320,11 +295,6 @@ public class JobUpdaterIT extends EasyMockTest {
         stateManager.insertPendingTasks(storeProvider, task, instanceIds));
   }
 
-  private ILock insertInProgressUpdate(IJobUpdate update) {
-    return storage.write(
-        storeProvider -> saveJobUpdate(storeProvider.getJobUpdateStore(), 
update, ROLLING_FORWARD));
-  }
-
   private void insertInitialTasks(IJobUpdate update) {
     storage.write((NoResult.Quiet) storeProvider -> {
       for (IInstanceTaskConfig config : 
update.getInstructions().getInitialState()) {
@@ -472,8 +442,8 @@ public class JobUpdaterIT extends EasyMockTest {
     changeState(JOB, 1, ASSIGNED, STARTING, RUNNING);
     clock.advance(ONE_DAY);
 
-    storage.write(
-        storeProvider -> saveJobUpdate(storeProvider.getJobUpdateStore(), 
update, ROLLING_FORWARD));
+    storage.write((NoResult.Quiet) storeProvider ->
+        saveJobUpdate(storeProvider.getJobUpdateStore(), update, 
ROLLING_FORWARD));
 
     clock.advance(ONE_MINUTE);
 
@@ -517,7 +487,7 @@ public class JobUpdaterIT extends EasyMockTest {
     changeState(JOB, 0, ASSIGNED, STARTING, RUNNING);
     clock.advance(ONE_DAY);
 
-    storage.write(storeProvider ->
+    storage.write((NoResult.Quiet) storeProvider ->
         saveJobUpdate(storeProvider.getJobUpdateStore(), update, 
ROLL_FORWARD_AWAITING_PULSE));
 
     // The first pulse comes after one minute
@@ -561,7 +531,7 @@ public class JobUpdaterIT extends EasyMockTest {
     changeState(JOB, 0, ASSIGNED, STARTING, RUNNING);
     clock.advance(ONE_DAY);
 
-    storage.write(storeProvider ->
+    storage.write((NoResult.Quiet) storeProvider ->
         saveJobUpdate(storeProvider.getJobUpdateStore(), update, 
ROLL_FORWARD_AWAITING_PULSE));
 
     subscriber.startAsync().awaitRunning();
@@ -592,7 +562,7 @@ public class JobUpdaterIT extends EasyMockTest {
     changeState(JOB, 0, ASSIGNED, STARTING, RUNNING);
     clock.advance(ONE_DAY);
 
-    storage.write(storeProvider ->
+    storage.write((NoResult.Quiet) storeProvider ->
         saveJobUpdate(storeProvider.getJobUpdateStore(), update, 
ROLL_FORWARD_PAUSED));
 
     subscriber.startAsync().awaitRunning();
@@ -727,8 +697,8 @@ public class JobUpdaterIT extends EasyMockTest {
     changeState(JOB, 1, ASSIGNED, STARTING, RUNNING);
     clock.advance(ONE_DAY);
 
-    storage.write(
-        storeProvider -> saveJobUpdate(storeProvider.getJobUpdateStore(), 
update, ROLLING_FORWARD));
+    storage.write((NoResult.Quiet) storeProvider ->
+        saveJobUpdate(storeProvider.getJobUpdateStore(), update, 
ROLLING_FORWARD));
 
     clock.advance(ONE_MINUTE);
 
@@ -740,7 +710,6 @@ public class JobUpdaterIT extends EasyMockTest {
 
     storage.write((NoResult.Quiet) storeProvider -> {
       storeProvider.getJobUpdateStore().deleteAllUpdatesAndEvents();
-      releaseAllLocks();
     });
 
     // The pulse still returns OK but the error is handled.
@@ -761,7 +730,6 @@ public class JobUpdaterIT extends EasyMockTest {
     // Missing kill expectation will trigger failure.
     shutdownCommand.execute();
     expectLastCall().andAnswer(() -> {
-      storage.write((NoResult.Quiet) storeProvider -> releaseAllLocks());
       throw new IllegalStateException("Expected shutdown triggered.");
     });
 
@@ -1083,39 +1051,6 @@ public class JobUpdaterIT extends EasyMockTest {
     assertJobState(JOB, ImmutableMap.of(0, NEW_CONFIG, 1, OLD_CONFIG, 2, 
OLD_CONFIG));
   }
 
-  private void releaseAllLocks() {
-    for (ILock lock : storage.read(p -> p.getLockStore().fetchLocks())) {
-      lockManager.releaseLock(lock.getKey().getJob());
-    }
-  }
-
-  @Test
-  public void testLostLock() throws Exception {
-    // Validates the 'write-only' nature of locks in relation to job updates.  
We are in backwards
-    // compatibility mode for LockStore, so it is not used to control behavior 
of updates.
-    // As a result, out-of-band removal of locks will not impact the progress 
of job updates.
-
-    expectTaskKilled();
-
-    control.replay();
-
-    IJobUpdate update = makeJobUpdate(
-        makeInstanceConfig(0, 2, OLD_CONFIG));
-    insertInitialTasks(update);
-
-    changeState(JOB, 0, ASSIGNED, STARTING, RUNNING);
-    changeState(JOB, 1, ASSIGNED, STARTING, RUNNING);
-    clock.advance(WATCH_TIMEOUT);
-
-    // Instance 0 is updated.
-    updater.start(update, AUDIT);
-    releaseAllLocks();
-    changeState(JOB, 0, KILLED);
-    ImmutableMultimap.Builder<Integer, JobUpdateAction> actions = 
ImmutableMultimap.builder();
-    actions.putAll(0, INSTANCE_UPDATING);
-    assertState(ROLLING_FORWARD, actions.build());
-  }
-
   private void expectInvalid(JobUpdate update)
       throws UpdateStateException, UpdateConfigurationException {
 
@@ -1168,7 +1103,6 @@ public class JobUpdaterIT extends EasyMockTest {
 
       JobUpdate builder = update.newBuilder();
       builder.getInstructions().getSettings().setUpdateGroupSize(0);
-      releaseAllLocks();
       saveJobUpdate(store, IJobUpdate.build(builder), ROLLING_FORWARD);
     });
 
@@ -1186,21 +1120,13 @@ public class JobUpdaterIT extends EasyMockTest {
     assertState(ERROR, actions.build());
   }
 
-  private ILock saveJobUpdate(
+  private void saveJobUpdate(
       JobUpdateStore.Mutable store,
       IJobUpdate update,
       JobUpdateStatus status) {
 
-    ILock lock;
-    try {
-      lock = lockManager.acquireLock(update.getSummary().getKey().getJob(), 
USER);
-    } catch (LockManager.LockException e) {
-      throw Throwables.propagate(e);
-    }
-
-    store.saveJobUpdate(update, Optional.of(lock.getToken()));
+    store.saveJobUpdate(update);
     saveJobUpdateEvent(store, update, status);
-    return lock;
   }
 
   private void saveJobUpdateEvent(
@@ -1228,8 +1154,8 @@ public class JobUpdaterIT extends EasyMockTest {
     changeState(JOB, 1, ASSIGNED, STARTING, RUNNING);
     clock.advance(ONE_DAY);
 
-    storage.write(
-        storeProvider -> saveJobUpdate(storeProvider.getJobUpdateStore(), 
update, ROLLING_FORWARD));
+    storage.write((NoResult.Quiet) storeProvider ->
+        saveJobUpdate(storeProvider.getJobUpdateStore(), update, 
ROLLING_FORWARD));
 
     subscriber.startAsync().awaitRunning();
 
@@ -1249,21 +1175,6 @@ public class JobUpdaterIT extends EasyMockTest {
   }
 
   @Test
-  public void testSystemResumeNoLock() throws Exception {
-    control.replay();
-
-    IJobUpdate update = setInstanceCount(makeJobUpdate(makeInstanceConfig(0, 
1, OLD_CONFIG)), 0);
-
-    storage.write((NoResult.Quiet) storeProvider -> {
-      ILock lock = saveJobUpdate(storeProvider.getJobUpdateStore(), update, 
ROLLING_FORWARD);
-      lockManager.releaseLock(lock.getKey().getJob());
-    });
-
-    subscriber.startAsync().awaitRunning();
-    assertState(ERROR, ImmutableMultimap.of());
-  }
-
-  @Test
   public void testImmediatelySuccessfulUpdate() throws Exception {
     control.replay();
 
@@ -1387,69 +1298,6 @@ public class JobUpdaterIT extends EasyMockTest {
     updater.pause(UPDATE_ID, AUDIT);
   }
 
-  @Test
-  public void testAbortAfterLostLock() throws Exception {
-    expectTaskKilled();
-
-    control.replay();
-
-    IJobUpdate update = makeJobUpdate(makeInstanceConfig(0, 2, OLD_CONFIG));
-    insertInitialTasks(update);
-
-    changeState(JOB, 0, ASSIGNED, STARTING, RUNNING);
-    clock.advance(WATCH_TIMEOUT);
-
-    ImmutableMultimap.Builder<Integer, JobUpdateAction> actions = 
ImmutableMultimap.builder();
-
-    updater.start(update, AUDIT);
-    actions.putAll(0, INSTANCE_UPDATING);
-    assertState(ROLLING_FORWARD, actions.build());
-    releaseAllLocks();
-    updater.abort(update.getSummary().getKey(), AUDIT);
-    clock.advance(WATCH_TIMEOUT);
-    assertState(ABORTED, actions.build());
-  }
-
-  @Test
-  public void testStartUpdateAfterPausedAndLockLost() throws Exception {
-    // Tests for regression of AURORA-1023, in which a user could paint 
themselves into a corner
-    // by starting an update, pausing it, and forcibly releasing the job lock. 
 The result in this
-    // behavior should be to prevent further job updates until the user aborts 
the first one.
-
-    expectTaskKilled();
-
-    control.replay();
-
-    IJobUpdate update = makeJobUpdate(makeInstanceConfig(0, 2, OLD_CONFIG));
-    insertInitialTasks(update);
-
-    changeState(JOB, 0, ASSIGNED, STARTING, RUNNING);
-    clock.advance(WATCH_TIMEOUT);
-
-    ImmutableMultimap.Builder<Integer, JobUpdateAction> actions = 
ImmutableMultimap.builder();
-
-    updater.start(update, AUDIT);
-    actions.putAll(0, INSTANCE_UPDATING);
-    assertState(ROLLING_FORWARD, actions.build());
-
-    updater.pause(update.getSummary().getKey(), AUDIT);
-    assertState(ROLL_FORWARD_PAUSED, actions.build());
-    clock.advance(WATCH_TIMEOUT);
-
-    releaseAllLocks();
-
-    JobUpdate builder = makeJobUpdate(makeInstanceConfig(0, 0, 
OLD_CONFIG)).newBuilder();
-    builder.getSummary().getKey().setId("another update");
-    IJobUpdate update2 = IJobUpdate.build(builder);
-
-    try {
-      updater.start(update2, AUDIT);
-      fail();
-    } catch (UpdateStateException e) {
-      // Expected.
-    }
-  }
-
   @Test(expected = UpdateStateException.class)
   public void testResumeUnknownUpdate() throws Exception {
     control.replay();
@@ -1661,8 +1509,8 @@ public class JobUpdaterIT extends EasyMockTest {
     control.replay();
 
     IJobUpdate inProgress = makeJobUpdate();
-    ILock lock = insertInProgressUpdate(inProgress);
-
+    storage.write((NoResult.Quiet) storeProvider ->
+        saveJobUpdate(storeProvider.getJobUpdateStore(), inProgress, 
ROLLING_FORWARD));
     IJobUpdate anotherUpdate = makeJobUpdate();
     try {
       updater.start(anotherUpdate, AUDIT);
@@ -1672,9 +1520,6 @@ public class JobUpdaterIT extends EasyMockTest {
       assertEquals(
           inProgress.getSummary().newBuilder().setState(new 
JobUpdateState(ROLLING_FORWARD, 0, 0)),
           e.getInProgressUpdateSummary().newBuilder());
-      assertEquals(ImmutableSet.of(lock), storage.read(p -> 
p.getLockStore().fetchLocks()));
-    } finally {
-      lockManager.releaseLock(lock.getKey().getJob());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/4fecf1f5/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 f61f73a..f2a2eae 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
@@ -40,7 +40,6 @@ from gen.apache.aurora.api.ttypes import (
     JobKey,
     JobUpdateQuery,
     JobUpdateRequest,
-    Lock,
     ResourceAggregate,
     Response,
     ResponseCode,
@@ -99,10 +98,9 @@ class TestSchedulerProxyInjection(unittest.TestCase):
 
   def test_replaceCronTemplate(self):
     self.mock_thrift_client.replaceCronTemplate(
-        IsA(JobConfiguration),
-        IsA(Lock)).AndReturn(DEFAULT_RESPONSE)
+        IsA(JobConfiguration)).AndReturn(DEFAULT_RESPONSE)
     self.mox.ReplayAll()
-    self.make_scheduler_proxy().replaceCronTemplate(JobConfiguration(), Lock())
+    self.make_scheduler_proxy().replaceCronTemplate(JobConfiguration())
 
   def test_scheduleCronJob(self):
     self.mock_thrift_client.scheduleCronJob(

Reply via email to