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

incubator-aurora git commit: Move read-modify-write logic out of AttributeStore.Mutable for simpler layering.

Repository: incubator-aurora
Updated Branches:
  refs/heads/master 64ffdc3da -> 0774c4e67


Move read-modify-write logic out of AttributeStore.Mutable for simpler layering.

Bugs closed: AURORA-913

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


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

Branch: refs/heads/master
Commit: 0774c4e67157a8b2b0cf92cf1c4f9388f825514a
Parents: 64ffdc3
Author: Bill Farner <wf...@apache.org>
Authored: Tue Nov 11 11:28:02 2014 -0800
Committer: Bill Farner <wf...@apache.org>
Committed: Tue Nov 11 11:28:02 2014 -0800

----------------------------------------------------------------------
 .../aurora/scheduler/http/Maintenance.java      | 13 ++--
 .../scheduler/mesos/MesosSchedulerImpl.java     |  6 +-
 .../scheduler/state/MaintenanceController.java  |  4 +-
 .../scheduler/storage/AttributeStore.java       | 63 ++++++++++++++++----
 .../scheduler/storage/db/DbAttributeStore.java  | 51 ++++------------
 .../storage/log/WriteAheadStorage.java          | 30 ++--------
 .../scheduler/mesos/MesosSchedulerImplTest.java | 29 ++++++++-
 .../state/MaintenanceControllerImplTest.java    | 11 +++-
 .../storage/db/DbAttributeStoreTest.java        | 40 ++++---------
 .../scheduler/storage/log/LogStorageTest.java   | 25 ++++----
 .../storage/log/SnapshotStoreImplTest.java      |  2 +-
 .../storage/log/WriteAheadStorageTest.java      | 37 ------------
 12 files changed, 138 insertions(+), 173 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/0774c4e6/src/main/java/org/apache/aurora/scheduler/http/Maintenance.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/http/Maintenance.java b/src/main/java/org/apache/aurora/scheduler/http/Maintenance.java
index a7a807d..be8a1ff 100644
--- a/src/main/java/org/apache/aurora/scheduler/http/Maintenance.java
+++ b/src/main/java/org/apache/aurora/scheduler/http/Maintenance.java
@@ -32,6 +32,7 @@ import com.google.common.collect.Multimaps;
 import org.apache.aurora.gen.MaintenanceMode;
 import org.apache.aurora.scheduler.base.Query;
 import org.apache.aurora.scheduler.base.Tasks;
+import org.apache.aurora.scheduler.storage.AttributeStore;
 import org.apache.aurora.scheduler.storage.Storage;
 import org.apache.aurora.scheduler.storage.Storage.StoreProvider;
 import org.apache.aurora.scheduler.storage.Storage.Work;
@@ -62,7 +63,9 @@ public class Maintenance {
       public Response apply(StoreProvider storeProvider) {
         Multimap<MaintenanceMode, String> hostsByMode =
             Multimaps.transformValues(
-              Multimaps.index(storeProvider.getAttributeStore().getHostAttributes(), GET_MODE),
+              Multimaps.index(
+                  storeProvider.getAttributeStore().getHostAttributes(),
+                  AttributeStore.Util.GET_MODE),
               HOST_NAME);
 
         Map<MaintenanceMode, Object> hosts = Maps.newHashMap();
@@ -89,12 +92,4 @@ public class Maintenance {
           return attributes.getHost();
         }
       };
-
-  private static final Function<IHostAttributes, MaintenanceMode> GET_MODE =
-      new Function<IHostAttributes, MaintenanceMode>() {
-        @Override
-        public MaintenanceMode apply(IHostAttributes attrs) {
-          return attrs.getMode();
-        }
-      };
 }

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/0774c4e6/src/main/java/org/apache/aurora/scheduler/mesos/MesosSchedulerImpl.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/mesos/MesosSchedulerImpl.java b/src/main/java/org/apache/aurora/scheduler/mesos/MesosSchedulerImpl.java
index 2d382f7..ffcbc97 100644
--- a/src/main/java/org/apache/aurora/scheduler/mesos/MesosSchedulerImpl.java
+++ b/src/main/java/org/apache/aurora/scheduler/mesos/MesosSchedulerImpl.java
@@ -31,11 +31,11 @@ import com.twitter.common.stats.Stats;
 
 import org.apache.aurora.GuiceUtils.AllowUnchecked;
 import org.apache.aurora.scheduler.TaskLauncher;
-import org.apache.aurora.scheduler.base.Conversions;
 import org.apache.aurora.scheduler.base.SchedulerException;
 import org.apache.aurora.scheduler.events.EventSink;
 import org.apache.aurora.scheduler.events.PubsubEvent.DriverDisconnected;
 import org.apache.aurora.scheduler.events.PubsubEvent.DriverRegistered;
+import org.apache.aurora.scheduler.storage.AttributeStore;
 import org.apache.aurora.scheduler.storage.Storage;
 import org.apache.aurora.scheduler.storage.Storage.MutableStoreProvider;
 import org.apache.aurora.scheduler.storage.Storage.MutateWork;
@@ -167,8 +167,8 @@ class MesosSchedulerImpl implements Scheduler {
           @Override
           protected void execute(MutableStoreProvider storeProvider) {
             for (final Offer offer : offers) {
-              storeProvider.getAttributeStore()
-                  .saveHostAttributes(Conversions.getAttributes(offer));
+              storeProvider.getAttributeStore().saveHostAttributes(
+                  AttributeStore.Util.mergeOffer(storeProvider.getAttributeStore(), offer));
             }
           }
         });

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/0774c4e6/src/main/java/org/apache/aurora/scheduler/state/MaintenanceController.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/state/MaintenanceController.java b/src/main/java/org/apache/aurora/scheduler/state/MaintenanceController.java
index 21cfebd..88418e6 100644
--- a/src/main/java/org/apache/aurora/scheduler/state/MaintenanceController.java
+++ b/src/main/java/org/apache/aurora/scheduler/state/MaintenanceController.java
@@ -268,7 +268,9 @@ public interface MaintenanceController {
       AttributeStore.Mutable store = storeProvider.getAttributeStore();
       ImmutableSet.Builder<HostStatus> statuses = ImmutableSet.builder();
       for (String host : hosts) {
-        if (store.setMaintenanceMode(host, mode)) {
+        Optional<IHostAttributes> toSave = AttributeStore.Util.mergeMode(store, host, mode);
+        if (toSave.isPresent()) {
+          store.saveHostAttributes(toSave.get());
           HostStatus status = new HostStatus().setHost(host).setMode(mode);
           eventSink.post(new PubsubEvent.HostMaintenanceStateChange(status.deepCopy()));
           statuses.add(status);

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/0774c4e6/src/main/java/org/apache/aurora/scheduler/storage/AttributeStore.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/AttributeStore.java b/src/main/java/org/apache/aurora/scheduler/storage/AttributeStore.java
index 8ca0dfd..97f9ee8 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/AttributeStore.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/AttributeStore.java
@@ -15,13 +15,16 @@ package org.apache.aurora.scheduler.storage;
 
 import java.util.Set;
 
+import com.google.common.base.Function;
 import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableList;
 
 import org.apache.aurora.gen.MaintenanceMode;
+import org.apache.aurora.scheduler.base.Conversions;
 import org.apache.aurora.scheduler.storage.Storage.StoreProvider;
 import org.apache.aurora.scheduler.storage.entities.IAttribute;
 import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
+import org.apache.mesos.Protos;
 
 /**
  * Storage interface for host attributes.
@@ -57,19 +60,10 @@ public interface AttributeStore {
      * Save a host attribute in the attribute store.
      *
      * @param hostAttributes The attribute we are going to save.
+     * @return {@code true} if the operation changed the attributes stored for the given
+     *         {@link IHostAttributes#getHost() host}, or {@code false} if the save was a no-op.
      */
-    void saveHostAttributes(IHostAttributes hostAttributes);
-
-    /**
-     * Adjusts the maintenance mode for a host.
-     * No adjustment will be made if the host is unknown.
-     *
-     * @param host Host to adjust.
-     * @param mode Mode to place the host in.
-     * @return {@code true} if the host is known and the state was adjusted,
-     *         {@code false} if the host is unrecognized.
-     */
-    boolean setMaintenanceMode(String host, MaintenanceMode mode);
+    boolean saveHostAttributes(IHostAttributes hostAttributes);
   }
 
   final class Util {
@@ -89,5 +83,50 @@ public interface AttributeStore {
       return attributes.isPresent()
           ? attributes.get().getAttributes() : ImmutableList.<IAttribute>of();
     }
+
+    /**
+     * Merges a desired maintenance mode with the existing attributes of a host, if present.
+     *
+     * @param store The store to fetch existing attributes from.
+     * @param host The host to merge existing attributes from.
+     * @param mode Maintenance mode to save if the host is known.
+     * @return The attributes that should be saved if there were already attributes stored for
+     *         the {@code host}, or {@link Optional#absent() none} if the host is unknown and
+     *         attributes should not be saved.
+     */
+    public static Optional<IHostAttributes> mergeMode(
+        AttributeStore store,
+        String host,
+        MaintenanceMode mode) {
+
+      Optional<IHostAttributes> stored = store.getHostAttributes(host);
+      if (stored.isPresent()) {
+        return Optional.of(IHostAttributes.build(stored.get().newBuilder().setMode(mode)));
+      } else {
+        return Optional.absent();
+      }
+    }
+
+    public static final Function<IHostAttributes, MaintenanceMode> GET_MODE =
+        new Function<IHostAttributes, MaintenanceMode>() {
+          @Override
+          public MaintenanceMode apply(IHostAttributes attrs) {
+            return attrs.getMode();
+          }
+        };
+
+    /**
+     * Merges the attributes from an offer, applying the default maintenance mode.
+     *
+     * @param store Store to fetch the existing maintenance mode for this host.
+     * @param offer Offer to merge.
+     * @return attributes from {@code offer} and the existing (or default) maintenance mode.
+     */
+    public static IHostAttributes mergeOffer(AttributeStore store, Protos.Offer offer) {
+      IHostAttributes fromOffer = Conversions.getAttributes(offer);
+      MaintenanceMode mode =
+          store.getHostAttributes(fromOffer.getHost()).transform(GET_MODE).or(MaintenanceMode.NONE);
+      return IHostAttributes.build(fromOffer.newBuilder().setMode(mode));
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/0774c4e6/src/main/java/org/apache/aurora/scheduler/storage/db/DbAttributeStore.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/DbAttributeStore.java b/src/main/java/org/apache/aurora/scheduler/storage/db/DbAttributeStore.java
index c8b3095..5f1cd2b 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/db/DbAttributeStore.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/DbAttributeStore.java
@@ -18,17 +18,15 @@ import java.util.Set;
 
 import com.google.common.base.Optional;
 import com.google.common.base.Predicate;
-import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
 import com.google.inject.Inject;
 
-import org.apache.aurora.gen.Attribute;
-import org.apache.aurora.gen.HostAttributes;
-import org.apache.aurora.gen.MaintenanceMode;
 import org.apache.aurora.scheduler.storage.AttributeStore;
 import org.apache.aurora.scheduler.storage.entities.IAttribute;
 import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.twitter.common.base.MorePreconditions.checkNotBlank;
 import static com.twitter.common.inject.TimedInterceptor.Timed;
 
 /**
@@ -50,35 +48,18 @@ class DbAttributeStore implements AttributeStore.Mutable {
 
   @Timed("attribute_store_save")
   @Override
-  public void saveHostAttributes(IHostAttributes hostAttributes) {
-    HostAttributes mutableAttributes = hostAttributes.newBuilder();
+  public boolean saveHostAttributes(IHostAttributes hostAttributes) {
+    checkNotBlank(hostAttributes.getHost());
+    checkArgument(hostAttributes.isSetAttributes());
+    checkArgument(hostAttributes.isSetMode());
 
-    // Default to NONE maintenance mode.
-    if (!hostAttributes.isSetMode()) {
-      mutableAttributes.setMode(MaintenanceMode.NONE);
-    }
-    // Ensure attributes is non-null.
-    if (!hostAttributes.isSetAttributes()) {
-      mutableAttributes.setAttributes(ImmutableSet.<Attribute>of());
-    }
-
-    // If this is an 'upsert', don't overwrite the previously-set maintenance mode.
     Optional<IHostAttributes> existing = getHostAttributes(hostAttributes.getHost());
-    IHostAttributes toSave;
-    if (existing.isPresent()) {
-      mutableAttributes.setMode(existing.get().getMode());
-
-      toSave = IHostAttributes.build(mutableAttributes);
-
-      // Avoid inserting again if this is a no-op update.
-      if (existing.get().equals(toSave)) {
-        return;
-      }
+    if (existing.equals(Optional.of(hostAttributes))) {
+      return false;
     } else {
-      toSave = IHostAttributes.build(mutableAttributes);
+      merge(hostAttributes);
+      return true;
     }
-
-    merge(toSave);
   }
 
   private static final Predicate<IAttribute> EMPTY_VALUES = new Predicate<IAttribute>() {
@@ -101,18 +82,6 @@ class DbAttributeStore implements AttributeStore.Mutable {
     }
   }
 
-  @Timed("attribute_store_set_mode")
-  @Override
-  public boolean setMaintenanceMode(String host, MaintenanceMode mode) {
-    Optional<IHostAttributes> existing = getHostAttributes(host);
-    if (existing.isPresent()) {
-      merge(IHostAttributes.build(existing.get().newBuilder().setMode(mode)));
-      return true;
-    } else {
-      return false;
-    }
-  }
-
   @Timed("attribute_store_fetch_one")
   @Override
   public Optional<IHostAttributes> getHostAttributes(String host) {

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/0774c4e6/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 5790b84..c6250b4 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
@@ -25,7 +25,6 @@ import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Maps;
 import com.twitter.common.base.MorePreconditions;
 
-import org.apache.aurora.gen.MaintenanceMode;
 import org.apache.aurora.gen.storage.Op;
 import org.apache.aurora.gen.storage.PruneJobUpdateHistory;
 import org.apache.aurora.gen.storage.RemoveJob;
@@ -211,20 +210,14 @@ class WriteAheadStorage extends ForwardingStore implements
   }
 
   @Override
-  public void saveHostAttributes(final IHostAttributes attrs) {
+  public boolean saveHostAttributes(final IHostAttributes attrs) {
     requireNonNull(attrs);
 
-    // Pass the updated attributes upstream, and then check if the stored value changes.
-    // We do this since different parts of the system write partial HostAttributes objects
-    // and they are merged together internally.
-    // TODO(William Farner): Split out a separate method
-    //                       saveAttributes(String host, Iterable<Attributes>) to simplify this.
-    Optional<IHostAttributes> saved = getHostAttributes(attrs.getHost());
-    attributeStore.saveHostAttributes(attrs);
-    Optional<IHostAttributes> updated = getHostAttributes(attrs.getHost());
-    if (!saved.equals(updated)) {
-      write(Op.saveHostAttributes(new SaveHostAttributes(updated.get().newBuilder())));
+    boolean changed = attributeStore.saveHostAttributes(attrs);
+    if (changed) {
+      write(Op.saveHostAttributes(new SaveHostAttributes(attrs.newBuilder())));
     }
+    return changed;
   }
 
   @Override
@@ -351,19 +344,6 @@ class WriteAheadStorage extends ForwardingStore implements
   }
 
   @Override
-  public boolean setMaintenanceMode(final String host, final MaintenanceMode mode) {
-    requireNonNull(host);
-    requireNonNull(mode);
-
-    boolean saved = attributeStore.setMaintenanceMode(host, mode);
-    if (saved) {
-      write(Op.saveHostAttributes(
-          new SaveHostAttributes(attributeStore.getHostAttributes(host).get().newBuilder())));
-    }
-    return saved;
-  }
-
-  @Override
   public SchedulerStore.Mutable getSchedulerStore() {
     return this;
   }

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/0774c4e6/src/test/java/org/apache/aurora/scheduler/mesos/MesosSchedulerImplTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/mesos/MesosSchedulerImplTest.java b/src/test/java/org/apache/aurora/scheduler/mesos/MesosSchedulerImplTest.java
index af15c95..052562f 100644
--- a/src/test/java/org/apache/aurora/scheduler/mesos/MesosSchedulerImplTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/mesos/MesosSchedulerImplTest.java
@@ -20,6 +20,7 @@ import java.util.List;
 import java.util.concurrent.Executor;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableList;
 import com.google.common.testing.TearDown;
 import com.google.common.util.concurrent.MoreExecutors;
@@ -31,6 +32,8 @@ import com.twitter.common.application.Lifecycle;
 import com.twitter.common.base.Command;
 import com.twitter.common.testing.easymock.EasyMockTest;
 
+import org.apache.aurora.gen.HostAttributes;
+import org.apache.aurora.gen.MaintenanceMode;
 import org.apache.aurora.scheduler.TaskLauncher;
 import org.apache.aurora.scheduler.base.Conversions;
 import org.apache.aurora.scheduler.base.SchedulerException;
@@ -39,6 +42,7 @@ import org.apache.aurora.scheduler.events.PubsubEvent.DriverDisconnected;
 import org.apache.aurora.scheduler.events.PubsubEvent.DriverRegistered;
 import org.apache.aurora.scheduler.storage.Storage;
 import org.apache.aurora.scheduler.storage.Storage.StorageException;
+import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 import org.apache.aurora.scheduler.storage.testing.StorageTestUtil;
 import org.apache.mesos.Protos.ExecutorID;
 import org.apache.mesos.Protos.FrameworkID;
@@ -53,6 +57,7 @@ import org.apache.mesos.SchedulerDriver;
 import org.junit.Before;
 import org.junit.Test;
 
+import static org.apache.aurora.gen.MaintenanceMode.DRAINING;
 import static org.easymock.EasyMock.expect;
 import static org.junit.Assert.assertTrue;
 
@@ -175,6 +180,24 @@ public class MesosSchedulerImplTest extends EasyMockTest {
   }
 
   @Test
+  public void testAttributesModePreserved() throws Exception {
+    new OfferFixture() {
+      @Override
+      void respondToOffer() throws Exception {
+        IHostAttributes draining = IHostAttributes.build(new HostAttributes().setMode(DRAINING));
+        expect(storageUtil.attributeStore.getHostAttributes(OFFER.getHostname()))
+            .andReturn(Optional.of(draining));
+        IHostAttributes saved = IHostAttributes.build(
+            Conversions.getAttributes(OFFER).newBuilder().setMode(DRAINING));
+        expect(storageUtil.attributeStore.saveHostAttributes(saved)).andReturn(true);
+
+        expect(systemLauncher.willUse(OFFER)).andReturn(false);
+        expect(userLauncher.willUse(OFFER)).andReturn(true);
+      }
+    }.run();
+  }
+
+  @Test
   public void testStatusUpdateNoAccepts() throws Exception {
     new StatusFixture() {
       @Override
@@ -264,7 +287,11 @@ public class MesosSchedulerImplTest extends EasyMockTest {
   }
 
   private void expectOfferAttributesSaved(Offer offer) {
-    storageUtil.attributeStore.saveHostAttributes(Conversions.getAttributes(offer));
+    expect(storageUtil.attributeStore.getHostAttributes(offer.getHostname()))
+        .andReturn(Optional.<IHostAttributes>absent());
+    IHostAttributes defaultMode = IHostAttributes.build(
+        Conversions.getAttributes(offer).newBuilder().setMode(MaintenanceMode.NONE));
+    expect(storageUtil.attributeStore.saveHostAttributes(defaultMode)).andReturn(true);
   }
 
   private abstract class RegisteredFixture {

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/0774c4e6/src/test/java/org/apache/aurora/scheduler/state/MaintenanceControllerImplTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/state/MaintenanceControllerImplTest.java b/src/test/java/org/apache/aurora/scheduler/state/MaintenanceControllerImplTest.java
index 4d96761..99fa11b 100644
--- a/src/test/java/org/apache/aurora/scheduler/state/MaintenanceControllerImplTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/state/MaintenanceControllerImplTest.java
@@ -126,8 +126,8 @@ public class MaintenanceControllerImplTest extends EasyMockTest {
 
   @Test
   public void testUnknownHost() {
-    expect(storageUtil.attributeStore.setMaintenanceMode("b", MaintenanceMode.SCHEDULED))
-        .andReturn(false);
+    expect(storageUtil.attributeStore.getHostAttributes("b"))
+        .andReturn(Optional.<IHostAttributes>absent());
 
     control.replay();
 
@@ -186,7 +186,12 @@ public class MaintenanceControllerImplTest extends EasyMockTest {
   }
 
   private void expectMaintenanceModeChange(String hostName, MaintenanceMode mode) {
-    expect(storageUtil.attributeStore.setMaintenanceMode(hostName, mode)).andReturn(true);
+    IHostAttributes attributes = IHostAttributes.build(new HostAttributes().setHost(hostName));
+
+    expect(storageUtil.attributeStore.getHostAttributes(hostName))
+        .andReturn(Optional.of(attributes));
+    IHostAttributes updated = IHostAttributes.build(attributes.newBuilder().setMode(mode));
+    expect(storageUtil.attributeStore.saveHostAttributes(updated)).andReturn(true);
     eventSink.post(
         new PubsubEvent.HostMaintenanceStateChange(
             new HostStatus().setHost(hostName).setMode(mode)));

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/0774c4e6/src/test/java/org/apache/aurora/scheduler/storage/db/DbAttributeStoreTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/storage/db/DbAttributeStoreTest.java b/src/test/java/org/apache/aurora/scheduler/storage/db/DbAttributeStoreTest.java
index c0a1aa5..d659658 100644
--- a/src/test/java/org/apache/aurora/scheduler/storage/db/DbAttributeStoreTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/storage/db/DbAttributeStoreTest.java
@@ -33,8 +33,6 @@ import org.junit.Test;
 
 import static org.apache.aurora.gen.MaintenanceMode.DRAINED;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
 
 public class DbAttributeStoreTest {
 
@@ -48,10 +46,12 @@ public class DbAttributeStoreTest {
   private static final IHostAttributes HOST_A_ATTRS =
       IHostAttributes.build(new HostAttributes(HOST_A, ImmutableSet.of(ATTR1, ATTR2))
           .setSlaveId(SLAVE_A)
+          .setAttributes(ImmutableSet.<Attribute>of())
           .setMode(MaintenanceMode.NONE));
   private static final IHostAttributes HOST_B_ATTRS =
       IHostAttributes.build(new HostAttributes(HOST_B, ImmutableSet.of(ATTR2, ATTR3))
           .setSlaveId(SLAVE_B)
+          .setAttributes(ImmutableSet.<Attribute>of())
           .setMode(MaintenanceMode.DRAINING));
 
   private Storage storage;
@@ -81,6 +81,11 @@ public class DbAttributeStoreTest {
     assertEquals(Optional.of(updatedA), read(HOST_A));
     assertEquals(ImmutableSet.of(updatedA, HOST_B_ATTRS), readAll());
 
+    IHostAttributes updatedMode = IHostAttributes.build(updatedA.newBuilder().setMode(DRAINED));
+    insert(updatedMode);
+    assertEquals(Optional.of(updatedMode), read(HOST_A));
+    assertEquals(ImmutableSet.of(updatedMode, HOST_B_ATTRS), readAll());
+
     truncate();
     assertEquals(Optional.<IHostAttributes>absent(), read(HOST_A));
     assertEquals(ImmutableSet.<IHostAttributes>of(), readAll());
@@ -101,36 +106,20 @@ public class DbAttributeStoreTest {
     assertEquals(Optional.of(attributes), read(HOST_A));
   }
 
-  @Test
-  public void testSetMaintenanceMode() {
+  @Test(expected = IllegalArgumentException.class)
+  public void testNoMode() {
     HostAttributes noMode = HOST_A_ATTRS.newBuilder();
     noMode.unsetMode();
 
     insert(IHostAttributes.build(noMode));
-    // Default mode NONE should be automatically applied.
-    assertEquals(Optional.of(HOST_A_ATTRS), read(HOST_A));
-
-    IHostAttributes updatedA = IHostAttributes.build(noMode.deepCopy().setMode(DRAINED));
-    // Inserting the updated value should ignore the mode.
-    insert(updatedA);
-    assertEquals(Optional.of(HOST_A_ATTRS), read(HOST_A));
-
-    // Instead, the mode must be explicitly set to be changed.
-    assertTrue(setMode(HOST_A, DRAINED));
-    assertEquals(Optional.of(updatedA), read(HOST_A));
-
-    assertFalse(setMode(HOST_B, DRAINED));
   }
 
-  @Test
+  @Test(expected = IllegalArgumentException.class)
   public void testSaveAttributesNotSet() {
     HostAttributes attributes = HOST_A_ATTRS.newBuilder();
     attributes.unsetAttributes();
 
     insert(IHostAttributes.build(attributes));
-    assertEquals(
-        Optional.of(IHostAttributes.build(attributes.setAttributes(ImmutableSet.<Attribute>of()))),
-        read(HOST_A));
   }
 
   @Test
@@ -150,15 +139,6 @@ public class DbAttributeStoreTest {
     });
   }
 
-  private boolean setMode(final String host, final MaintenanceMode mode) {
-    return storage.write(new MutateWork.Quiet<Boolean>() {
-      @Override
-      public Boolean apply(MutableStoreProvider storeProvider) {
-        return storeProvider.getAttributeStore().setMaintenanceMode(host, mode);
-      }
-    });
-  }
-
   private Optional<IHostAttributes> read(final String host) {
     return storage.consistentRead(new Work.Quiet<Optional<IHostAttributes>>() {
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/0774c4e6/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 55a4784..c6ff43f 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
@@ -115,6 +115,7 @@ import static org.easymock.EasyMock.expect;
 import static org.easymock.EasyMock.expectLastCall;
 import static org.easymock.EasyMock.notNull;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 public class LogStorageTest extends EasyMockTest {
@@ -329,8 +330,8 @@ public class LogStorageTest extends EasyMockTest {
         .setSlaveId("slave2")
         .setMode(MaintenanceMode.DRAINED));
     builder.add(createTransaction(Op.saveHostAttributes(hostAttributes2)));
-    storageUtil.attributeStore.saveHostAttributes(
-        IHostAttributes.build(hostAttributes2.getHostAttributes()));
+    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)));
@@ -837,26 +838,30 @@ public class LogStorageTest extends EasyMockTest {
         expect(storageUtil.attributeStore.getHostAttributes(host))
             .andReturn(Optional.<IHostAttributes>absent());
 
-        // Each logStorage save invokes get, save, get to the underlying attribute store.
-        storageUtil.attributeStore.saveHostAttributes(hostAttributes.get());
         expect(storageUtil.attributeStore.getHostAttributes(host)).andReturn(hostAttributes);
+
+        expect(storageUtil.attributeStore.saveHostAttributes(hostAttributes.get())).andReturn(true);
         streamMatcher.expectTransaction(
             Op.saveHostAttributes(new SaveHostAttributes(hostAttributes.get().newBuilder())))
             .andReturn(position);
-        expect(storageUtil.attributeStore.getHostAttributes(host)).andReturn(hostAttributes);
 
-        expect(storageUtil.attributeStore.getHostAttributes(host)).andReturn(hostAttributes);
-        storageUtil.attributeStore.saveHostAttributes(hostAttributes.get());
-        expect(storageUtil.attributeStore.getHostAttributes(host)).andReturn(hostAttributes);
+        expect(storageUtil.attributeStore.saveHostAttributes(hostAttributes.get()))
+            .andReturn(false);
+
         expect(storageUtil.attributeStore.getHostAttributes(host)).andReturn(hostAttributes);
       }
 
       @Override
       protected void performMutations(MutableStoreProvider storeProvider) {
         AttributeStore.Mutable store = storeProvider.getAttributeStore();
-        store.saveHostAttributes(hostAttributes.get());
+        assertEquals(Optional.<IHostAttributes>absent(), store.getHostAttributes(host));
+
+        assertTrue(store.saveHostAttributes(hostAttributes.get()));
+
         assertEquals(hostAttributes, store.getHostAttributes(host));
-        store.saveHostAttributes(hostAttributes.get());
+
+        assertFalse(store.saveHostAttributes(hostAttributes.get()));
+
         assertEquals(hostAttributes, store.getHostAttributes(host));
       }
     }.run();

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/0774c4e6/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplTest.java b/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplTest.java
index ab0d501..8fc3cb8 100644
--- a/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplTest.java
@@ -130,7 +130,7 @@ public class SnapshotStoreImplTest extends EasyMockTest {
     expectDataWipe();
     storageUtil.taskStore.saveTasks(tasks);
     storageUtil.quotaStore.saveQuota("steve", ResourceAggregates.none());
-    storageUtil.attributeStore.saveHostAttributes(attribute);
+    expect(storageUtil.attributeStore.saveHostAttributes(attribute)).andReturn(true);
     storageUtil.jobStore.saveAcceptedJob(
         job.getJobManagerId(),
         IJobConfiguration.build(job.getJobConfiguration()));

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/0774c4e6/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 792c9fe..a553c56 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
@@ -18,18 +18,13 @@ import java.util.logging.Level;
 import java.util.logging.Logger;
 
 import com.google.common.base.Function;
-import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableSet;
 import com.twitter.common.testing.easymock.EasyMockTest;
 
 import org.apache.aurora.gen.AssignedTask;
-import org.apache.aurora.gen.Attribute;
-import org.apache.aurora.gen.HostAttributes;
-import org.apache.aurora.gen.MaintenanceMode;
 import org.apache.aurora.gen.ScheduledTask;
 import org.apache.aurora.gen.storage.Op;
 import org.apache.aurora.gen.storage.PruneJobUpdateHistory;
-import org.apache.aurora.gen.storage.SaveHostAttributes;
 import org.apache.aurora.gen.storage.SaveTasks;
 import org.apache.aurora.scheduler.base.Query;
 import org.apache.aurora.scheduler.storage.AttributeStore;
@@ -39,7 +34,6 @@ 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;
-import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 import org.apache.aurora.scheduler.storage.entities.IScheduledTask;
 import org.easymock.EasyMock;
 import org.junit.Before;
@@ -47,8 +41,6 @@ import org.junit.Test;
 
 import static org.easymock.EasyMock.expect;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
 
 public class WriteAheadStorageTest extends EasyMockTest {
 
@@ -93,35 +85,6 @@ public class WriteAheadStorageTest extends EasyMockTest {
   }
 
   @Test
-  public void testSetMaintenanceModeSaved() {
-    String host = "a";
-    MaintenanceMode mode = MaintenanceMode.DRAINED;
-    expect(attributeStore.setMaintenanceMode(host, mode)).andReturn(true);
-    IHostAttributes attribute = IHostAttributes.build(new HostAttributes()
-        .setHost(host)
-        .setMode(mode)
-        .setAttributes(ImmutableSet.of(
-            new Attribute().setName("os").setValues(ImmutableSet.of("linux")))));
-    expect(attributeStore.getHostAttributes(host)).andReturn(Optional.of(attribute));
-    expectOp(Op.saveHostAttributes(new SaveHostAttributes(attribute.newBuilder())));
-
-    control.replay();
-
-    assertTrue(storage.setMaintenanceMode(host, mode));
-  }
-
-  @Test
-  public void testSetMaintenanceModeNotSaved() {
-    String host = "a";
-    MaintenanceMode mode = MaintenanceMode.DRAINED;
-    expect(attributeStore.setMaintenanceMode(host, mode)).andReturn(false);
-
-    control.replay();
-
-    assertFalse(storage.setMaintenanceMode(host, mode));
-  }
-
-  @Test
   public void testPruneHistory() {
     Set<String> pruned = ImmutableSet.of("a", "b");
     expect(jobUpdateStore.pruneHistory(1, 1)).andReturn(pruned);