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/06/26 22:59:09 UTC

git commit: Use immutable types in AttributeStore.

Repository: incubator-aurora
Updated Branches:
  refs/heads/master 06656cc14 -> 7011c8332


Use immutable types in AttributeStore.

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


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

Branch: refs/heads/master
Commit: 7011c8332268295e20ed035356916c19e2500f11
Parents: 06656cc
Author: Bill Farner <wf...@apache.org>
Authored: Thu Jun 26 13:42:39 2014 -0700
Committer: Bill Farner <wf...@apache.org>
Committed: Thu Jun 26 13:42:39 2014 -0700

----------------------------------------------------------------------
 .../org/apache/aurora/scheduler/TaskVars.java   | 21 +++++++------
 .../aurora/scheduler/base/Conversions.java      |  8 ++---
 .../scheduler/filter/AttributeAggregate.java    | 12 ++++----
 .../scheduler/filter/AttributeFilter.java       |  5 +--
 .../scheduler/filter/ConstraintFilter.java      | 22 ++++++++------
 .../scheduler/filter/SchedulingFilterImpl.java  |  8 ++---
 .../aurora/scheduler/http/Maintenance.java      | 14 ++++-----
 .../apache/aurora/scheduler/http/Slaves.java    | 32 ++++++++++----------
 .../scheduler/state/MaintenanceController.java  | 17 ++++++-----
 .../scheduler/storage/AttributeStore.java       | 16 +++++-----
 .../scheduler/storage/ForwardingStore.java      |  6 ++--
 .../scheduler/storage/log/LogStorage.java       |  4 ++-
 .../storage/log/SnapshotStoreImpl.java          |  6 ++--
 .../storage/log/WriteAheadStorage.java          | 22 ++++++--------
 .../storage/mem/MemAttributeStore.java          | 26 ++++++++++------
 .../apache/aurora/scheduler/TaskVarsTest.java   |  7 +++--
 .../scheduler/async/PreemptorImplTest.java      |  6 ++--
 .../filter/AttributeAggregateTest.java          |  7 +++--
 .../filter/SchedulingFilterImplTest.java        |  7 +++--
 .../state/MaintenanceControllerImplTest.java    | 17 ++++++-----
 .../scheduler/storage/log/LogStorageTest.java   | 12 +++++---
 .../storage/log/SnapshotStoreImplTest.java      |  7 +++--
 22 files changed, 152 insertions(+), 130 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/7011c833/src/main/java/org/apache/aurora/scheduler/TaskVars.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/TaskVars.java b/src/main/java/org/apache/aurora/scheduler/TaskVars.java
index 75e7917..a58696b 100644
--- a/src/main/java/org/apache/aurora/scheduler/TaskVars.java
+++ b/src/main/java/org/apache/aurora/scheduler/TaskVars.java
@@ -32,7 +32,6 @@ import com.google.common.collect.Iterables;
 import com.google.common.eventbus.Subscribe;
 import com.twitter.common.stats.StatsProvider;
 
-import org.apache.aurora.gen.Attribute;
 import org.apache.aurora.gen.ScheduleStatus;
 import org.apache.aurora.scheduler.events.PubsubEvent.EventSubscriber;
 import org.apache.aurora.scheduler.events.PubsubEvent.SchedulerActive;
@@ -42,6 +41,7 @@ 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;
+import org.apache.aurora.scheduler.storage.entities.IAttribute;
 import org.apache.aurora.scheduler.storage.entities.IScheduledTask;
 
 import static com.google.common.base.Preconditions.checkNotNull;
@@ -82,19 +82,20 @@ class TaskVars implements EventSubscriber {
     return "tasks_lost_rack_" + rack;
   }
 
-  private static final Predicate<Attribute> IS_RACK = new Predicate<Attribute>() {
+  private static final Predicate<IAttribute> IS_RACK = new Predicate<IAttribute>() {
     @Override
-    public boolean apply(Attribute attr) {
+    public boolean apply(IAttribute attr) {
       return "rack".equals(attr.getName());
     }
   };
 
-  private static final Function<Attribute, String> ATTR_VALUE = new Function<Attribute, String>() {
-    @Override
-    public String apply(Attribute attr) {
-      return Iterables.getOnlyElement(attr.getValues());
-    }
-  };
+  private static final Function<IAttribute, String> ATTR_VALUE =
+      new Function<IAttribute, String>() {
+        @Override
+        public String apply(IAttribute attr) {
+          return Iterables.getOnlyElement(attr.getValues());
+        }
+      };
 
   private Counter getCounter(ScheduleStatus status) {
     return counters.getUnchecked(getVarName(status));
@@ -123,7 +124,7 @@ class TaskVars implements EventSubscriber {
       Optional<String> rack = storage.consistentRead(new Work.Quiet<Optional<String>>() {
         @Override
         public Optional<String> apply(StoreProvider storeProvider) {
-          Optional<Attribute> rack = FluentIterable
+          Optional<IAttribute> rack = FluentIterable
               .from(AttributeStore.Util.attributesOrNone(storeProvider, host))
               .firstMatch(IS_RACK);
           return rack.transform(ATTR_VALUE);

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/7011c833/src/main/java/org/apache/aurora/scheduler/base/Conversions.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/base/Conversions.java b/src/main/java/org/apache/aurora/scheduler/base/Conversions.java
index 02e57ac..7b7650d 100644
--- a/src/main/java/org/apache/aurora/scheduler/base/Conversions.java
+++ b/src/main/java/org/apache/aurora/scheduler/base/Conversions.java
@@ -30,6 +30,7 @@ import org.apache.aurora.gen.Attribute;
 import org.apache.aurora.gen.HostAttributes;
 import org.apache.aurora.gen.ScheduleStatus;
 import org.apache.aurora.scheduler.configuration.ConfigurationManager;
+import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 import org.apache.mesos.Protos;
 import org.apache.mesos.Protos.Offer;
 import org.apache.mesos.Protos.TaskState;
@@ -120,18 +121,17 @@ public final class Conversions {
    * @param offer Resource offer.
    * @return Equivalent thrift host attributes.
    */
-  public static HostAttributes getAttributes(Offer offer) {
+  public static IHostAttributes getAttributes(Offer offer) {
     // Group by attribute name.
     Multimap<String, Protos.Attribute> valuesByName =
         Multimaps.index(offer.getAttributesList(), ATTRIBUTE_NAME);
 
-    // TODO(William Farner): Include slave id.
-    return new HostAttributes(
+    return IHostAttributes.build(new HostAttributes(
         offer.getHostname(),
         FluentIterable.from(valuesByName.asMap().entrySet())
             .transform(ATTRIBUTE_CONVERTER)
             .toSet())
-        .setSlaveId(offer.getSlaveId().getValue());
+        .setSlaveId(offer.getSlaveId().getValue()));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/7011c833/src/main/java/org/apache/aurora/scheduler/filter/AttributeAggregate.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/filter/AttributeAggregate.java b/src/main/java/org/apache/aurora/scheduler/filter/AttributeAggregate.java
index 1afc6ca..71620d7 100644
--- a/src/main/java/org/apache/aurora/scheduler/filter/AttributeAggregate.java
+++ b/src/main/java/org/apache/aurora/scheduler/filter/AttributeAggregate.java
@@ -26,8 +26,8 @@ import com.google.common.collect.Iterables;
 import com.google.common.util.concurrent.AtomicLongMap;
 import com.twitter.common.collections.Pair;
 
-import org.apache.aurora.gen.Attribute;
 import org.apache.aurora.scheduler.storage.AttributeStore;
+import org.apache.aurora.scheduler.storage.entities.IAttribute;
 import org.apache.aurora.scheduler.storage.entities.IScheduledTask;
 
 import static com.google.common.base.Preconditions.checkNotNull;
@@ -63,10 +63,10 @@ public class AttributeAggregate {
     checkNotNull(activeTaskSupplier);
     checkNotNull(attributeStore);
 
-    final Function<IScheduledTask, Iterable<Attribute>> getHostAttributes =
-        new Function<IScheduledTask, Iterable<Attribute>>() {
+    final Function<IScheduledTask, Iterable<IAttribute>> getHostAttributes =
+        new Function<IScheduledTask, Iterable<IAttribute>>() {
           @Override
-          public Iterable<Attribute> apply(IScheduledTask task) {
+          public Iterable<IAttribute> apply(IScheduledTask task) {
             // Note: this assumes we have access to attributes for hosts where all active tasks
             // reside.
             String host = checkNotNull(task.getAssignedTask().getSlaveHost());
@@ -78,9 +78,9 @@ public class AttributeAggregate {
       @Override
       public Map<Pair<String, String>, Long> get() {
         AtomicLongMap<Pair<String, String>> counts = AtomicLongMap.create();
-        Iterable<Attribute> allAttributes =
+        Iterable<IAttribute> allAttributes =
             Iterables.concat(Iterables.transform(activeTaskSupplier.get(), getHostAttributes));
-        for (Attribute attribute : allAttributes) {
+        for (IAttribute attribute : allAttributes) {
           for (String value : attribute.getValues()) {
             counts.incrementAndGet(Pair.of(attribute.getName(), value));
           }

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/7011c833/src/main/java/org/apache/aurora/scheduler/filter/AttributeFilter.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/filter/AttributeFilter.java b/src/main/java/org/apache/aurora/scheduler/filter/AttributeFilter.java
index 06e402d..88efda0 100644
--- a/src/main/java/org/apache/aurora/scheduler/filter/AttributeFilter.java
+++ b/src/main/java/org/apache/aurora/scheduler/filter/AttributeFilter.java
@@ -18,7 +18,7 @@ import java.util.Set;
 import com.google.common.base.Predicates;
 import com.google.common.collect.Iterables;
 
-import org.apache.aurora.gen.Attribute;
+import org.apache.aurora.scheduler.storage.entities.IAttribute;
 import org.apache.aurora.scheduler.storage.entities.IValueConstraint;
 
 /**
@@ -50,7 +50,8 @@ final class AttributeFilter {
    * @param attributeAggregate Cached state of the job being filtered.
    * @return {@code true} if the limit constraint is satisfied, {@code false} otherwise.
    */
-  static boolean matches(final Attribute attribute,
+  static boolean matches(
+      final IAttribute attribute,
       int limit,
       AttributeAggregate attributeAggregate) {
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/7011c833/src/main/java/org/apache/aurora/scheduler/filter/ConstraintFilter.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/filter/ConstraintFilter.java b/src/main/java/org/apache/aurora/scheduler/filter/ConstraintFilter.java
index 0a76e43..860a240 100644
--- a/src/main/java/org/apache/aurora/scheduler/filter/ConstraintFilter.java
+++ b/src/main/java/org/apache/aurora/scheduler/filter/ConstraintFilter.java
@@ -25,6 +25,7 @@ import com.google.common.collect.Iterables;
 import org.apache.aurora.gen.Attribute;
 import org.apache.aurora.scheduler.base.SchedulerException;
 import org.apache.aurora.scheduler.filter.SchedulingFilter.Veto;
+import org.apache.aurora.scheduler.storage.entities.IAttribute;
 import org.apache.aurora.scheduler.storage.entities.IConstraint;
 import org.apache.aurora.scheduler.storage.entities.ITaskConstraint;
 
@@ -35,7 +36,7 @@ import static com.google.common.base.Preconditions.checkNotNull;
  */
 class ConstraintFilter {
   private final AttributeAggregate cachedjobState;
-  private final Iterable<Attribute> hostAttributes;
+  private final Iterable<IAttribute> hostAttributes;
 
   /**
    * Creates a new constraint filer for a given job.
@@ -43,7 +44,7 @@ class ConstraintFilter {
    * @param cachedjobState Cached information about the job containing the task being matched.
    * @param hostAttributes The attributes of the host to test against.
    */
-  ConstraintFilter(AttributeAggregate cachedjobState, Iterable<Attribute> hostAttributes) {
+  ConstraintFilter(AttributeAggregate cachedjobState, Iterable<IAttribute> hostAttributes) {
     this.cachedjobState = checkNotNull(cachedjobState);
     this.hostAttributes = checkNotNull(hostAttributes);
   }
@@ -63,10 +64,10 @@ class ConstraintFilter {
     return new Veto("Host " + reason + " for maintenance", Veto.MAX_SCORE);
   }
 
-  private static final Function<Attribute, Set<String>> GET_VALUES =
-      new Function<Attribute, Set<String>>() {
+  private static final Function<IAttribute, Set<String>> GET_VALUES =
+      new Function<IAttribute, Set<String>>() {
         @Override
-        public Set<String> apply(Attribute attribute) {
+        public Set<String> apply(IAttribute attribute) {
           return attribute.getValues();
         }
       };
@@ -79,15 +80,16 @@ class ConstraintFilter {
    * @return A veto if the constraint is not satisfied based on the existing state of the job.
    */
   Optional<Veto> getVeto(IConstraint constraint) {
-    Iterable<Attribute> sameNameAttributes =
+    Iterable<IAttribute> sameNameAttributes =
         Iterables.filter(hostAttributes, new NameFilter(constraint.getName()));
-    Optional<Attribute> attribute;
+    Optional<IAttribute> attribute;
     if (Iterables.isEmpty(sameNameAttributes)) {
       attribute = Optional.absent();
     } else {
       Set<String> attributeValues = ImmutableSet.copyOf(
           Iterables.concat(Iterables.transform(sameNameAttributes, GET_VALUES)));
-      attribute = Optional.of(new Attribute(constraint.getName(), attributeValues));
+      attribute =
+          Optional.of(IAttribute.build(new Attribute(constraint.getName(), attributeValues)));
     }
 
     ITaskConstraint taskConstraint = constraint.getConstraint();
@@ -122,7 +124,7 @@ class ConstraintFilter {
   /**
    * A filter to find attributes matching a name.
    */
-  static class NameFilter implements Predicate<Attribute> {
+  static class NameFilter implements Predicate<IAttribute> {
     private final String attributeName;
 
     NameFilter(String attributeName) {
@@ -130,7 +132,7 @@ class ConstraintFilter {
     }
 
     @Override
-    public boolean apply(Attribute attribute) {
+    public boolean apply(IAttribute attribute) {
       return attributeName.equals(attribute.getName());
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/7011c833/src/main/java/org/apache/aurora/scheduler/filter/SchedulingFilterImpl.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/filter/SchedulingFilterImpl.java b/src/main/java/org/apache/aurora/scheduler/filter/SchedulingFilterImpl.java
index 589cab3..a336b47 100644
--- a/src/main/java/org/apache/aurora/scheduler/filter/SchedulingFilterImpl.java
+++ b/src/main/java/org/apache/aurora/scheduler/filter/SchedulingFilterImpl.java
@@ -29,7 +29,6 @@ import com.google.common.collect.Ordering;
 import com.twitter.common.quantity.Amount;
 import com.twitter.common.quantity.Data;
 
-import org.apache.aurora.gen.Attribute;
 import org.apache.aurora.gen.MaintenanceMode;
 import org.apache.aurora.gen.TaskConstraint;
 import org.apache.aurora.scheduler.ResourceSlot;
@@ -39,6 +38,7 @@ 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.Quiet;
+import org.apache.aurora.scheduler.storage.entities.IAttribute;
 import org.apache.aurora.scheduler.storage.entities.IConstraint;
 import org.apache.aurora.scheduler.storage.entities.ITaskConfig;
 
@@ -246,10 +246,10 @@ public class SchedulingFilterImpl implements SchedulingFilter {
   }
 
   private boolean isDedicated(final String slaveHost) {
-    Iterable<Attribute> slaveAttributes =
-        storage.weaklyConsistentRead(new Quiet<Iterable<Attribute>>() {
+    Iterable<IAttribute> slaveAttributes =
+        storage.weaklyConsistentRead(new Quiet<Iterable<IAttribute>>() {
           @Override
-          public Iterable<Attribute> apply(final StoreProvider storeProvider) {
+          public Iterable<IAttribute> apply(final StoreProvider storeProvider) {
             return AttributeStore.Util.attributesOrNone(storeProvider, slaveHost);
           }
         });

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/7011c833/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 d780ed6..5470e6f 100644
--- a/src/main/java/org/apache/aurora/scheduler/http/Maintenance.java
+++ b/src/main/java/org/apache/aurora/scheduler/http/Maintenance.java
@@ -29,13 +29,13 @@ import com.google.common.collect.Maps;
 import com.google.common.collect.Multimap;
 import com.google.common.collect.Multimaps;
 
-import org.apache.aurora.gen.HostAttributes;
 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.Storage;
 import org.apache.aurora.scheduler.storage.Storage.StoreProvider;
 import org.apache.aurora.scheduler.storage.Storage.Work;
+import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 import org.apache.aurora.scheduler.storage.entities.IScheduledTask;
 
 import static org.apache.aurora.gen.MaintenanceMode.DRAINED;
@@ -82,18 +82,18 @@ public class Maintenance {
         Tasks.SCHEDULED_TO_ID);
   }
 
-  private static final Function<HostAttributes, String> HOST_NAME =
-      new Function<HostAttributes, String>() {
+  private static final Function<IHostAttributes, String> HOST_NAME =
+      new Function<IHostAttributes, String>() {
         @Override
-        public String apply(HostAttributes attributes) {
+        public String apply(IHostAttributes attributes) {
           return attributes.getHost();
         }
       };
 
-  private static final Function<HostAttributes, MaintenanceMode> GET_MODE =
-      new Function<HostAttributes, MaintenanceMode>() {
+  private static final Function<IHostAttributes, MaintenanceMode> GET_MODE =
+      new Function<IHostAttributes, MaintenanceMode>() {
         @Override
-        public MaintenanceMode apply(HostAttributes attrs) {
+        public MaintenanceMode apply(IHostAttributes attrs) {
           return attrs.getMode();
         }
       };

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/7011c833/src/main/java/org/apache/aurora/scheduler/http/Slaves.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/http/Slaves.java b/src/main/java/org/apache/aurora/scheduler/http/Slaves.java
index 4c26db4..bbbdb92 100644
--- a/src/main/java/org/apache/aurora/scheduler/http/Slaves.java
+++ b/src/main/java/org/apache/aurora/scheduler/http/Slaves.java
@@ -28,10 +28,10 @@ import com.google.common.collect.Ordering;
 import com.twitter.common.base.Closure;
 
 import org.antlr.stringtemplate.StringTemplate;
-import org.apache.aurora.gen.Attribute;
-import org.apache.aurora.gen.HostAttributes;
 import org.apache.aurora.gen.MaintenanceMode;
 import org.apache.aurora.scheduler.storage.Storage;
+import org.apache.aurora.scheduler.storage.entities.IAttribute;
+import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 import org.apache.aurora.scheduler.storage.entities.IServerInfo;
 
 import static com.google.common.base.Preconditions.checkNotNull;
@@ -61,19 +61,19 @@ public class Slaves extends JerseyTemplateServlet {
     this.storage = checkNotNull(storage);
   }
 
-  private Iterable<HostAttributes> getHostAttributes() {
-    return storage.weaklyConsistentRead(new Work.Quiet<Iterable<HostAttributes>>() {
+  private Iterable<IHostAttributes> getHostAttributes() {
+    return storage.weaklyConsistentRead(new Work.Quiet<Iterable<IHostAttributes>>() {
       @Override
-      public Iterable<HostAttributes> apply(StoreProvider storeProvider) {
+      public Iterable<IHostAttributes> apply(StoreProvider storeProvider) {
         return storeProvider.getAttributeStore().getHostAttributes();
       }
     });
   }
 
-  private static final Function<HostAttributes, Slave> TO_SLAVE =
-      new Function<HostAttributes, Slave>() {
+  private static final Function<IHostAttributes, Slave> TO_SLAVE =
+      new Function<IHostAttributes, Slave>() {
         @Override
-        public Slave apply(HostAttributes attributes) {
+        public Slave apply(IHostAttributes attributes) {
           return new Slave(attributes);
         }
       };
@@ -97,10 +97,10 @@ public class Slaves extends JerseyTemplateServlet {
     });
   }
 
-  private static final Ordering<Attribute> ATTR_ORDER = Ordering.natural().onResultOf(
-      new Function<Attribute, String>() {
+  private static final Ordering<IAttribute> ATTR_ORDER = Ordering.natural().onResultOf(
+      new Function<IAttribute, String>() {
         @Override
-        public String apply(Attribute attr) {
+        public String apply(IAttribute attr) {
           return attr .getName();
         }
       });
@@ -109,9 +109,9 @@ public class Slaves extends JerseyTemplateServlet {
    * Template object to represent a slave.
    */
   private static class Slave {
-    private final HostAttributes attributes;
+    private final IHostAttributes attributes;
 
-    Slave(HostAttributes attributes) {
+    Slave(IHostAttributes attributes) {
       this.attributes = attributes;
     }
 
@@ -127,10 +127,10 @@ public class Slaves extends JerseyTemplateServlet {
       return attributes.getMode();
     }
 
-    private static final Function<Attribute, String> ATTR_TO_STRING =
-        new Function<Attribute, String>() {
+    private static final Function<IAttribute, String> ATTR_TO_STRING =
+        new Function<IAttribute, String>() {
           @Override
-          public String apply(Attribute attr) {
+          public String apply(IAttribute attr) {
             return attr.getName() + "=[" + Joiner.on(",").join(attr.getValues()) + "]";
           }
         };

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/7011c833/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 332598f..0c3e1fb 100644
--- a/src/main/java/org/apache/aurora/scheduler/state/MaintenanceController.java
+++ b/src/main/java/org/apache/aurora/scheduler/state/MaintenanceController.java
@@ -26,7 +26,6 @@ import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Sets;
 import com.google.common.eventbus.Subscribe;
 
-import org.apache.aurora.gen.HostAttributes;
 import org.apache.aurora.gen.HostStatus;
 import org.apache.aurora.gen.MaintenanceMode;
 import org.apache.aurora.gen.ScheduleStatus;
@@ -42,6 +41,7 @@ import org.apache.aurora.scheduler.storage.Storage.MutableStoreProvider;
 import org.apache.aurora.scheduler.storage.Storage.MutateWork;
 import org.apache.aurora.scheduler.storage.Storage.StoreProvider;
 import org.apache.aurora.scheduler.storage.Storage.Work;
+import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 
 import static com.google.common.base.Preconditions.checkNotNull;
 
@@ -157,7 +157,8 @@ public interface MaintenanceController {
           public void execute(MutableStoreProvider store) {
             // If the task _was_ associated with a draining host, and it was the last task on the
             // host.
-            Optional<HostAttributes> attributes = store.getAttributeStore().getHostAttributes(host);
+            Optional<IHostAttributes> attributes =
+                store.getAttributeStore().getHostAttributes(host);
             if (attributes.isPresent() && attributes.get().getMode() == DRAINING) {
               Query.Builder builder = Query.slaveScoped(host).active();
               if (store.getTaskStore().fetchTasks(builder).isEmpty()) {
@@ -193,18 +194,18 @@ public interface MaintenanceController {
       });
     }
 
-    private static final Function<HostAttributes, String> HOST_NAME =
-        new Function<HostAttributes, String>() {
+    private static final Function<IHostAttributes, String> HOST_NAME =
+        new Function<IHostAttributes, String>() {
           @Override
-          public String apply(HostAttributes attributes) {
+          public String apply(IHostAttributes attributes) {
             return attributes.getHost();
           }
         };
 
-    private static final Function<HostAttributes, HostStatus> ATTRS_TO_STATUS =
-        new Function<HostAttributes, HostStatus>() {
+    private static final Function<IHostAttributes, HostStatus> ATTRS_TO_STATUS =
+        new Function<IHostAttributes, HostStatus>() {
           @Override
-          public HostStatus apply(HostAttributes attributes) {
+          public HostStatus apply(IHostAttributes attributes) {
             return new HostStatus().setHost(attributes.getHost()).setMode(attributes.getMode());
           }
         };

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/7011c833/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 6aa3e1b..8ca0dfd 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/AttributeStore.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/AttributeStore.java
@@ -18,10 +18,10 @@ import java.util.Set;
 import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableList;
 
-import org.apache.aurora.gen.Attribute;
-import org.apache.aurora.gen.HostAttributes;
 import org.apache.aurora.gen.MaintenanceMode;
 import org.apache.aurora.scheduler.storage.Storage.StoreProvider;
+import org.apache.aurora.scheduler.storage.entities.IAttribute;
+import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 
 /**
  * Storage interface for host attributes.
@@ -33,14 +33,14 @@ public interface AttributeStore {
    * @param host host name.
    * @return attributes associated with {@code host}, if the host is known.
    */
-  Optional<HostAttributes> getHostAttributes(String host);
+  Optional<IHostAttributes> getHostAttributes(String host);
 
   /**
    * Fetches all attributes in the store.
    *
    * @return All host attributes.
    */
-  Set<HostAttributes> getHostAttributes();
+  Set<IHostAttributes> getHostAttributes();
 
   /**
    * Attributes are considered mostly ephemeral and extremely low risk when inconsistency
@@ -58,7 +58,7 @@ public interface AttributeStore {
      *
      * @param hostAttributes The attribute we are going to save.
      */
-    void saveHostAttributes(HostAttributes hostAttributes);
+    void saveHostAttributes(IHostAttributes hostAttributes);
 
     /**
      * Adjusts the maintenance mode for a host.
@@ -84,10 +84,10 @@ public interface AttributeStore {
      * @return Attributes associated with {@code host}, or an empty iterable if the host is
      *         unknown.
      */
-    public static Iterable<Attribute> attributesOrNone(StoreProvider store, String host) {
-      Optional<HostAttributes> attributes = store.getAttributeStore().getHostAttributes(host);
+    public static Iterable<IAttribute> attributesOrNone(StoreProvider store, String host) {
+      Optional<IHostAttributes> attributes = store.getAttributeStore().getHostAttributes(host);
       return attributes.isPresent()
-          ? attributes.get().getAttributes() : ImmutableList.<Attribute>of();
+          ? attributes.get().getAttributes() : ImmutableList.<IAttribute>of();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/7011c833/src/main/java/org/apache/aurora/scheduler/storage/ForwardingStore.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/ForwardingStore.java b/src/main/java/org/apache/aurora/scheduler/storage/ForwardingStore.java
index 6b0068c..e23506b 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/ForwardingStore.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/ForwardingStore.java
@@ -21,8 +21,8 @@ import javax.annotation.Nullable;
 import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableSet;
 
-import org.apache.aurora.gen.HostAttributes;
 import org.apache.aurora.scheduler.base.Query;
+import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 import org.apache.aurora.scheduler.storage.entities.IJobConfiguration;
 import org.apache.aurora.scheduler.storage.entities.IJobKey;
 import org.apache.aurora.scheduler.storage.entities.ILock;
@@ -124,12 +124,12 @@ public class ForwardingStore implements
   }
 
   @Override
-  public Optional<HostAttributes> getHostAttributes(String host) {
+  public Optional<IHostAttributes> getHostAttributes(String host) {
     return attributeStore.getHostAttributes(host);
   }
 
   @Override
-  public Set<HostAttributes> getHostAttributes() {
+  public Set<IHostAttributes> getHostAttributes() {
     return attributeStore.getHostAttributes();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/7011c833/src/main/java/org/apache/aurora/scheduler/storage/log/LogStorage.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/log/LogStorage.java b/src/main/java/org/apache/aurora/scheduler/storage/log/LogStorage.java
index 39f4712..2d7e81c 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/log/LogStorage.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/log/LogStorage.java
@@ -56,6 +56,7 @@ import org.apache.aurora.scheduler.storage.SnapshotStore;
 import org.apache.aurora.scheduler.storage.Storage;
 import org.apache.aurora.scheduler.storage.Storage.NonVolatileStorage;
 import org.apache.aurora.scheduler.storage.TaskStore;
+import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 import org.apache.aurora.scheduler.storage.entities.IJobConfiguration;
 import org.apache.aurora.scheduler.storage.entities.IJobKey;
 import org.apache.aurora.scheduler.storage.entities.ILock;
@@ -428,7 +429,8 @@ public class LogStorage implements NonVolatileStorage, DistributedSnapshotStore
         break;
 
       case SAVE_HOST_ATTRIBUTES:
-        writeBehindAttributeStore.saveHostAttributes(op.getSaveHostAttributes().hostAttributes);
+        writeBehindAttributeStore.saveHostAttributes(
+            IHostAttributes.build(op.getSaveHostAttributes().hostAttributes));
         break;
 
       case SAVE_LOCK:

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/7011c833/src/main/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl.java b/src/main/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl.java
index 0b4e22a..083e3ff 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl.java
@@ -39,6 +39,7 @@ import org.apache.aurora.scheduler.storage.Storage.MutateWork;
 import org.apache.aurora.scheduler.storage.Storage.StoreProvider;
 import org.apache.aurora.scheduler.storage.Storage.Volatile;
 import org.apache.aurora.scheduler.storage.Storage.Work;
+import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 import org.apache.aurora.scheduler.storage.entities.IJobConfiguration;
 import org.apache.aurora.scheduler.storage.entities.ILock;
 import org.apache.aurora.scheduler.storage.entities.IResourceAggregate;
@@ -59,7 +60,8 @@ public class SnapshotStoreImpl implements SnapshotStore<Snapshot> {
   private static final SnapshotField ATTRIBUTE_FIELD = new SnapshotField() {
     @Override
     public void saveToSnapshot(StoreProvider storeProvider, Snapshot snapshot) {
-      snapshot.setHostAttributes(storeProvider.getAttributeStore().getHostAttributes());
+      snapshot.setHostAttributes(
+          IHostAttributes.toBuildersSet(storeProvider.getAttributeStore().getHostAttributes()));
     }
 
     @Override
@@ -68,7 +70,7 @@ public class SnapshotStoreImpl implements SnapshotStore<Snapshot> {
 
       if (snapshot.isSetHostAttributes()) {
         for (HostAttributes attributes : snapshot.getHostAttributes()) {
-          store.getAttributeStore().saveHostAttributes(attributes);
+          store.getAttributeStore().saveHostAttributes(IHostAttributes.build(attributes));
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/7011c833/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 796dca3..4257cb9 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.inject.TimedInterceptor.Timed;
 
-import org.apache.aurora.gen.HostAttributes;
 import org.apache.aurora.gen.MaintenanceMode;
 import org.apache.aurora.gen.storage.Op;
 import org.apache.aurora.gen.storage.RemoveJob;
@@ -49,6 +48,7 @@ import org.apache.aurora.scheduler.storage.QuotaStore;
 import org.apache.aurora.scheduler.storage.SchedulerStore;
 import org.apache.aurora.scheduler.storage.Storage.MutableStoreProvider;
 import org.apache.aurora.scheduler.storage.TaskStore;
+import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 import org.apache.aurora.scheduler.storage.entities.IJobConfiguration;
 import org.apache.aurora.scheduler.storage.entities.IJobKey;
 import org.apache.aurora.scheduler.storage.entities.ILock;
@@ -197,7 +197,7 @@ class WriteAheadStorage extends ForwardingStore implements
 
   @Timed("scheduler_save_host_attribute")
   @Override
-  public void saveHostAttributes(final HostAttributes attrs) {
+  public void saveHostAttributes(final IHostAttributes attrs) {
     checkNotNull(attrs);
 
     // Pass the updated attributes upstream, and then check if the stored value changes.
@@ -205,11 +205,11 @@ class WriteAheadStorage extends ForwardingStore implements
     // and they are merged together internally.
     // TODO(William Farner): Split out a separate method
     //                       saveAttributes(String host, Iterable<Attributes>) to simplify this.
-    Optional<HostAttributes> saved = getHostAttributes(attrs.getHost());
+    Optional<IHostAttributes> saved = getHostAttributes(attrs.getHost());
     attributeStore.saveHostAttributes(attrs);
-    Optional<HostAttributes> updated = getHostAttributes(attrs.getHost());
+    Optional<IHostAttributes> updated = getHostAttributes(attrs.getHost());
     if (!saved.equals(updated)) {
-      write(Op.saveHostAttributes(new SaveHostAttributes(updated.get())));
+      write(Op.saveHostAttributes(new SaveHostAttributes(updated.get().newBuilder())));
     }
   }
 
@@ -294,14 +294,12 @@ class WriteAheadStorage extends ForwardingStore implements
     checkNotNull(host);
     checkNotNull(mode);
 
-    Optional<HostAttributes> saved = getHostAttributes(host);
-    if (saved.isPresent()) {
-      HostAttributes attributes = saved.get().setMode(mode);
-      write(Op.saveHostAttributes(new SaveHostAttributes(attributes)));
-      attributeStore.saveHostAttributes(attributes);
-      return true;
+    boolean saved = attributeStore.setMaintenanceMode(host, mode);
+    if (saved) {
+      write(Op.saveHostAttributes(
+          new SaveHostAttributes(attributeStore.getHostAttributes(host).get().newBuilder())));
     }
-    return false;
+    return saved;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/7011c833/src/main/java/org/apache/aurora/scheduler/storage/mem/MemAttributeStore.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/mem/MemAttributeStore.java b/src/main/java/org/apache/aurora/scheduler/storage/mem/MemAttributeStore.java
index ff9e45c..4bb807c 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/mem/MemAttributeStore.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/mem/MemAttributeStore.java
@@ -24,12 +24,13 @@ 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.Mutable;
+import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 
 /**
  * An in-memory attribute store.
  */
 class MemAttributeStore implements Mutable {
-  private final ConcurrentMap<String, HostAttributes> hostAttributes = Maps.newConcurrentMap();
+  private final ConcurrentMap<String, IHostAttributes> hostAttributes = Maps.newConcurrentMap();
 
   @Override
   public void deleteHostAttributes() {
@@ -37,35 +38,40 @@ class MemAttributeStore implements Mutable {
   }
 
   @Override
-  public void saveHostAttributes(HostAttributes attributes) {
+  public void saveHostAttributes(IHostAttributes attributes) {
     hostAttributes.putIfAbsent(attributes.getHost(), attributes);
 
-    HostAttributes stored = hostAttributes.get(attributes.getHost());
+    IHostAttributes stored = hostAttributes.get(attributes.getHost());
+    HostAttributes updated = stored.newBuilder();
     if (!stored.isSetMode()) {
-      stored.setMode(attributes.isSetMode() ? attributes.getMode() : MaintenanceMode.NONE);
+      updated.setMode(attributes.isSetMode() ? attributes.getMode() : MaintenanceMode.NONE);
     }
-    stored.setAttributes(attributes.isSetAttributes()
-        ? attributes.getAttributes() : ImmutableSet.<Attribute>of());
+    updated.setAttributes(updated.isSetAttributes()
+        ? updated.getAttributes() : ImmutableSet.<Attribute>of());
+    hostAttributes.replace(attributes.getHost(), stored, IHostAttributes.build(updated));
   }
 
   @Override
   public boolean setMaintenanceMode(String host, MaintenanceMode mode) {
-    HostAttributes stored = hostAttributes.get(host);
+    IHostAttributes stored = hostAttributes.get(host);
     if (stored == null) {
       return false;
     } else {
-      stored.setMode(mode);
+      hostAttributes.replace(
+          host,
+          stored,
+          IHostAttributes.build(stored.newBuilder().setMode(mode)));
       return true;
     }
   }
 
   @Override
-  public Optional<HostAttributes> getHostAttributes(String host) {
+  public Optional<IHostAttributes> getHostAttributes(String host) {
     return Optional.fromNullable(hostAttributes.get(host));
   }
 
   @Override
-  public Set<HostAttributes> getHostAttributes() {
+  public Set<IHostAttributes> getHostAttributes() {
     return ImmutableSet.copyOf(hostAttributes.values());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/7011c833/src/test/java/org/apache/aurora/scheduler/TaskVarsTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/TaskVarsTest.java b/src/test/java/org/apache/aurora/scheduler/TaskVarsTest.java
index da4eefa..2e128f4 100644
--- a/src/test/java/org/apache/aurora/scheduler/TaskVarsTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/TaskVarsTest.java
@@ -33,6 +33,7 @@ import org.apache.aurora.gen.TaskConfig;
 import org.apache.aurora.scheduler.events.PubsubEvent.SchedulerActive;
 import org.apache.aurora.scheduler.events.PubsubEvent.TaskStateChange;
 import org.apache.aurora.scheduler.events.PubsubEvent.TasksDeleted;
+import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 import org.apache.aurora.scheduler.storage.entities.IScheduledTask;
 import org.apache.aurora.scheduler.storage.testing.StorageTestUtil;
 import org.easymock.EasyMock;
@@ -199,10 +200,10 @@ public class TaskVarsTest extends EasyMockTest {
   }
 
   private IExpectationSetters<?> expectGetHostRack(String host, String rackToReturn) {
-    HostAttributes attributes = new HostAttributes()
+    IHostAttributes attributes = IHostAttributes.build(new HostAttributes()
         .setHost(host)
         .setAttributes(ImmutableSet.of(
-            new Attribute().setName("rack").setValues(ImmutableSet.of(rackToReturn))));
+            new Attribute().setName("rack").setValues(ImmutableSet.of(rackToReturn)))));
     return expect(storageUtil.attributeStore.getHostAttributes(host))
         .andReturn(Optional.of(attributes));
   }
@@ -238,7 +239,7 @@ public class TaskVarsTest extends EasyMockTest {
   public void testRackMissing() {
     expectStatusCountersInitialized();
     expect(storageUtil.attributeStore.getHostAttributes("a"))
-        .andReturn(Optional.<HostAttributes>absent());
+        .andReturn(Optional.<IHostAttributes>absent());
 
     control.replay();
     schedulerActivated();

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/7011c833/src/test/java/org/apache/aurora/scheduler/async/PreemptorImplTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/async/PreemptorImplTest.java b/src/test/java/org/apache/aurora/scheduler/async/PreemptorImplTest.java
index 3578bb3..8ee84cd 100644
--- a/src/test/java/org/apache/aurora/scheduler/async/PreemptorImplTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/async/PreemptorImplTest.java
@@ -52,6 +52,7 @@ 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;
+import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 import org.apache.aurora.scheduler.storage.entities.IScheduledTask;
 import org.apache.aurora.scheduler.storage.entities.ITaskConfig;
 import org.apache.aurora.scheduler.storage.mem.MemStorage;
@@ -627,8 +628,9 @@ public class PreemptorImplTest extends EasyMockTest {
 
   // Sets up a normal host, no dedicated hosts and no maintenance.
   private void setUpHost(String host, String rack) {
-    HostAttributes hostAttrs = new HostAttributes().setHost(host).setSlaveId(host + "_id")
-        .setMode(NONE).setAttributes(ImmutableSet.of(rack(rack), host(host)));
+    IHostAttributes hostAttrs = IHostAttributes.build(
+        new HostAttributes().setHost(host).setSlaveId(host + "_id")
+            .setMode(NONE).setAttributes(ImmutableSet.of(rack(rack), host(host))));
 
     expect(this.storageUtil.attributeStore.getHostAttributes(host))
         .andReturn(Optional.of(hostAttrs)).anyTimes();

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/7011c833/src/test/java/org/apache/aurora/scheduler/filter/AttributeAggregateTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/filter/AttributeAggregateTest.java b/src/test/java/org/apache/aurora/scheduler/filter/AttributeAggregateTest.java
index e2d9298..4b56576 100644
--- a/src/test/java/org/apache/aurora/scheduler/filter/AttributeAggregateTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/filter/AttributeAggregateTest.java
@@ -27,6 +27,7 @@ import org.apache.aurora.gen.Attribute;
 import org.apache.aurora.gen.HostAttributes;
 import org.apache.aurora.gen.ScheduledTask;
 import org.apache.aurora.scheduler.storage.AttributeStore;
+import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 import org.apache.aurora.scheduler.storage.entities.IScheduledTask;
 import org.easymock.IExpectationSetters;
 import org.junit.Before;
@@ -61,7 +62,7 @@ public class AttributeAggregateTest extends EasyMockTest {
   @Test(expected = IllegalStateException.class)
   public void testAttributesMissing() {
     expectGetTasks(task("1", "a"));
-    expect(attributeStore.getHostAttributes("a")).andReturn(Optional.<HostAttributes>absent());
+    expect(attributeStore.getHostAttributes("a")).andReturn(Optional.<IHostAttributes>absent());
 
     control.replay();
 
@@ -147,9 +148,9 @@ public class AttributeAggregateTest extends EasyMockTest {
 
   private IExpectationSetters<?> expectGetAttributes(String host, Attribute... attributes) {
     return expect(attributeStore.getHostAttributes(host)).andReturn(Optional.of(
-        new HostAttributes()
+        IHostAttributes.build(new HostAttributes()
             .setHost(host)
-            .setAttributes(ImmutableSet.<Attribute>builder().add(attributes).build())));
+            .setAttributes(ImmutableSet.<Attribute>builder().add(attributes).build()))));
   }
 
   private void assertAggregates(Map<Pair<String, String>, Long> expected) {

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/7011c833/src/test/java/org/apache/aurora/scheduler/filter/SchedulingFilterImplTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/filter/SchedulingFilterImplTest.java b/src/test/java/org/apache/aurora/scheduler/filter/SchedulingFilterImplTest.java
index c2c05cd..6a9c4ee 100644
--- a/src/test/java/org/apache/aurora/scheduler/filter/SchedulingFilterImplTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/filter/SchedulingFilterImplTest.java
@@ -45,6 +45,7 @@ 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.Quiet;
+import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 import org.apache.aurora.scheduler.storage.entities.IScheduledTask;
 import org.apache.aurora.scheduler.storage.entities.ITaskConfig;
 import org.easymock.EasyMock;
@@ -591,13 +592,13 @@ public class SchedulingFilterImplTest extends EasyMockTest {
     return expect(maintenance.getMode(host)).andReturn(mode);
   }
 
-  private IExpectationSetters<Optional<HostAttributes>> expectGetHostAttributes(
+  private IExpectationSetters<Optional<IHostAttributes>> expectGetHostAttributes(
       String host,
       Attribute... attributes) {
 
-    HostAttributes hostAttributes = new HostAttributes()
+    IHostAttributes hostAttributes = IHostAttributes.build(new HostAttributes()
         .setHost(host)
-        .setAttributes(ImmutableSet.<Attribute>builder().add(attributes).build());
+        .setAttributes(ImmutableSet.<Attribute>builder().add(attributes).build()));
     return expect(attributeStore.getHostAttributes(host)).andReturn(Optional.of(hostAttributes));
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/7011c833/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 4656317..4d96761 100644
--- a/src/test/java/org/apache/aurora/scheduler/state/MaintenanceControllerImplTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/state/MaintenanceControllerImplTest.java
@@ -36,6 +36,7 @@ import org.apache.aurora.scheduler.events.EventSink;
 import org.apache.aurora.scheduler.events.PubsubEvent;
 import org.apache.aurora.scheduler.events.PubsubEvent.TaskStateChange;
 import org.apache.aurora.scheduler.storage.Storage;
+import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 import org.apache.aurora.scheduler.storage.entities.IScheduledTask;
 import org.apache.aurora.scheduler.storage.testing.StorageTestUtil;
 import org.junit.Before;
@@ -99,7 +100,7 @@ public class MaintenanceControllerImplTest extends EasyMockTest {
     ScheduledTask task = makeTask(HOST_A, "taskA");
 
     expectMaintenanceModeChange(HOST_A, SCHEDULED);
-    expectFetchTasksByHost(HOST_A, ImmutableSet.<ScheduledTask>of(task));
+    expectFetchTasksByHost(HOST_A, ImmutableSet.of(task));
     expect(stateManager.changeState(
         Tasks.id(task),
         Optional.<ScheduleStatus>absent(),
@@ -107,8 +108,8 @@ public class MaintenanceControllerImplTest extends EasyMockTest {
         MaintenanceControllerImpl.DRAINING_MESSAGE))
         .andReturn(true);
     expectMaintenanceModeChange(HOST_A, DRAINING);
-    expect(storageUtil.attributeStore.getHostAttributes(HOST_A))
-        .andReturn(Optional.of(new HostAttributes().setHost(HOST_A).setMode(DRAINING)));
+    expect(storageUtil.attributeStore.getHostAttributes(HOST_A)).andReturn(Optional.of(
+        IHostAttributes.build(new HostAttributes().setHost(HOST_A).setMode(DRAINING))));
     // TaskA is FINISHED and therefore no longer active
     expectFetchTasksByHost(HOST_A, ImmutableSet.<ScheduledTask>of());
     expectMaintenanceModeChange(HOST_A, DRAINED);
@@ -150,8 +151,8 @@ public class MaintenanceControllerImplTest extends EasyMockTest {
   public void testEndEarly() {
     expectMaintenanceModeChange(HOST_A, SCHEDULED);
     expectMaintenanceModeChange(HOST_A, NONE);
-    expect(storageUtil.attributeStore.getHostAttributes(HOST_A))
-        .andReturn(Optional.of(new HostAttributes().setHost(HOST_A).setMode(NONE)));
+    expect(storageUtil.attributeStore.getHostAttributes(HOST_A)).andReturn(Optional.of(
+        IHostAttributes.build(new HostAttributes().setHost(HOST_A).setMode(NONE))));
 
     control.replay();
 
@@ -168,10 +169,10 @@ public class MaintenanceControllerImplTest extends EasyMockTest {
 
   @Test
   public void testGetMode() {
-    expect(storageUtil.attributeStore.getHostAttributes(HOST_A))
-        .andReturn(Optional.of(new HostAttributes().setHost(HOST_A).setMode(DRAINING)));
+    expect(storageUtil.attributeStore.getHostAttributes(HOST_A)).andReturn(Optional.of(
+        IHostAttributes.build(new HostAttributes().setHost(HOST_A).setMode(DRAINING))));
     expect(storageUtil.attributeStore.getHostAttributes("unknown"))
-        .andReturn(Optional.<HostAttributes>absent());
+        .andReturn(Optional.<IHostAttributes>absent());
 
     control.replay();
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/7011c833/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 53e5749..9ec00bb 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
@@ -70,6 +70,7 @@ import org.apache.aurora.scheduler.storage.Storage;
 import org.apache.aurora.scheduler.storage.Storage.MutableStoreProvider;
 import org.apache.aurora.scheduler.storage.Storage.MutateWork;
 import org.apache.aurora.scheduler.storage.Storage.MutateWork.NoResult;
+import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 import org.apache.aurora.scheduler.storage.entities.IJobConfiguration;
 import org.apache.aurora.scheduler.storage.entities.IJobKey;
 import org.apache.aurora.scheduler.storage.entities.ILock;
@@ -677,22 +678,23 @@ public class LogStorageTest extends EasyMockTest {
     final String host = "hostname";
     final Set<Attribute> attributes =
         ImmutableSet.of(new Attribute().setName("attr").setValues(ImmutableSet.of("value")));
-    final Optional<HostAttributes> hostAttributes = Optional.of(new HostAttributes()
-        .setHost(host)
-        .setAttributes(attributes));
+    final Optional<IHostAttributes> hostAttributes = Optional.of(
+        IHostAttributes.build(new HostAttributes()
+            .setHost(host)
+            .setAttributes(attributes)));
 
     new MutationFixture() {
       @Override
       protected void setupExpectations() throws Exception {
         storageUtil.expectWriteOperation();
         expect(storageUtil.attributeStore.getHostAttributes(host))
-            .andReturn(Optional.<HostAttributes>absent());
+            .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);
         streamMatcher.expectTransaction(
-            Op.saveHostAttributes(new SaveHostAttributes(hostAttributes.get())))
+            Op.saveHostAttributes(new SaveHostAttributes(hostAttributes.get().newBuilder())))
             .andReturn(position);
         expect(storageUtil.attributeStore.getHostAttributes(host)).andReturn(hostAttributes);
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/7011c833/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 5763fd0..22a4048 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
@@ -36,6 +36,7 @@ import org.apache.aurora.scheduler.base.JobKeys;
 import org.apache.aurora.scheduler.base.Query;
 import org.apache.aurora.scheduler.base.ResourceAggregates;
 import org.apache.aurora.scheduler.storage.SnapshotStore;
+import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 import org.apache.aurora.scheduler.storage.entities.IJobConfiguration;
 import org.apache.aurora.scheduler.storage.entities.ILock;
 import org.apache.aurora.scheduler.storage.entities.IScheduledTask;
@@ -69,8 +70,8 @@ public class SnapshotStoreImplTest extends EasyMockTest {
     Set<QuotaConfiguration> quotas =
         ImmutableSet.of(
             new QuotaConfiguration("steve", ResourceAggregates.none().newBuilder()));
-    HostAttributes attribute = new HostAttributes("host",
-        ImmutableSet.of(new Attribute("attr", ImmutableSet.of("value"))));
+    IHostAttributes attribute = IHostAttributes.build(new HostAttributes("host",
+        ImmutableSet.of(new Attribute("attr", ImmutableSet.of("value")))));
     StoredJob job = new StoredJob(
         "jobManager",
         new JobConfiguration().setKey(new JobKey("owner", "env", "name")));
@@ -111,7 +112,7 @@ public class SnapshotStoreImplTest extends EasyMockTest {
         .setTimestamp(NOW)
         .setTasks(IScheduledTask.toBuildersSet(tasks))
         .setQuotaConfigurations(quotas)
-        .setHostAttributes(ImmutableSet.of(attribute))
+        .setHostAttributes(ImmutableSet.of(attribute.newBuilder()))
         .setJobs(ImmutableSet.of(job))
         .setSchedulerMetadata(metadata)
         .setLocks(ILock.toBuildersSet(ImmutableSet.of(lock)));