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/07/18 00:05:51 UTC

git commit: Database-backed implementation of AttributeStore.

Repository: incubator-aurora
Updated Branches:
  refs/heads/master 3f6fba7e9 -> 5cf760bf3


Database-backed implementation of AttributeStore.

Bugs closed: AURORA-557

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


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

Branch: refs/heads/master
Commit: 5cf760bf31315c220c0f17cc233ad3a1dcfb6d86
Parents: 3f6fba7
Author: Bill Farner <wf...@apache.org>
Authored: Thu Jul 17 14:51:36 2014 -0700
Committer: Bill Farner <wf...@apache.org>
Committed: Thu Jul 17 14:51:36 2014 -0700

----------------------------------------------------------------------
 .../scheduler/storage/db/AttributeMapper.java   |  69 +++++++
 .../scheduler/storage/db/DbAttributeStore.java  | 112 +++++++++++
 .../aurora/scheduler/storage/db/DbModule.java   |   7 +
 .../aurora/scheduler/storage/db/DbStorage.java  |  24 ++-
 .../scheduler/storage/db/EnumValueMapper.java   |  31 ++++
 .../scheduler/storage/db/MigrationModule.java   |   2 +
 .../typehandlers/AbstractTEnumTypeHandler.java  |  62 +++++++
 .../MaintenanceModeTypeHandler.java             |  26 +++
 .../storage/db/typehandlers/TypeHandlers.java   |  33 ++++
 .../storage/mem/MemAttributeStore.java          |  89 ---------
 .../scheduler/storage/mem/MemStorage.java       |   2 +-
 .../scheduler/storage/mem/MemStorageModule.java |   2 -
 .../scheduler/storage/db/AttributeMapper.xml    |  81 ++++++++
 .../scheduler/storage/db/EnumValueMapper.xml    |  15 ++
 .../aurora/scheduler/storage/db/LockMapper.xml  |   2 +-
 .../aurora/scheduler/storage/db/schema.sql      |  27 +++
 .../storage/db/DbAttributeStoreTest.java        | 184 +++++++++++++++++++
 .../storage/mem/MemAttributeStoreTest.java      |  67 -------
 18 files changed, 671 insertions(+), 164 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/5cf760bf/src/main/java/org/apache/aurora/scheduler/storage/db/AttributeMapper.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/AttributeMapper.java b/src/main/java/org/apache/aurora/scheduler/storage/db/AttributeMapper.java
new file mode 100644
index 0000000..3763f4d
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/AttributeMapper.java
@@ -0,0 +1,69 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.aurora.scheduler.storage.db;
+
+import java.util.List;
+
+import javax.annotation.Nullable;
+
+import org.apache.aurora.gen.HostAttributes;
+import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
+
+/**
+ * MyBatis mapper interface for Attribute.xml.
+ */
+interface AttributeMapper {
+  /**
+   * Saves attributes for a host, based on {@link IHostAttributes#getHost()}.
+   *
+   * @param attributes Host attributes to save.
+   */
+  void insert(IHostAttributes attributes);
+
+  /**
+   * Deletes all attributes and attribute values associated with a slave.
+   *
+   * @param slaveId Slave ID to delete associated values from.
+   */
+  void deleteAttributesAndValues(String slaveId);
+
+  /**
+   * Inserts values in {@link IHostAttributes#getAttributes()}, associating them with
+   * {@link IHostAttributes#getSlaveId()}.
+   *
+   * @param attributes Attributes containing values to insert.
+   */
+  void insertAttributeValues(IHostAttributes attributes);
+
+  /**
+   * Retrieves the host attributes associated with a host.
+   *
+   * @param host Host to fetch attributes for.
+   * @return Attributes associated with {@code host}, or {@code null} if no association exists.
+   */
+  @Nullable
+  HostAttributes select(String host);
+
+  /**
+   * Retrieves all stored host attributes.
+   *
+   * @return All host attributes.
+   */
+  List<HostAttributes> selectAll();
+
+  /**
+   * Deletes all stored attributes and values.
+   */
+  void truncate();
+}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/5cf760bf/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
new file mode 100644
index 0000000..c5bfa1b
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/DbAttributeStore.java
@@ -0,0 +1,112 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.aurora.scheduler.storage.db;
+
+import java.util.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.checkNotNull;
+
+/**
+ * Attribute store backed by a relational database.
+ */
+class DbAttributeStore implements AttributeStore.Mutable {
+
+  private final AttributeMapper mapper;
+
+  @Inject
+  DbAttributeStore(AttributeMapper mapper) {
+    this.mapper = checkNotNull(mapper);
+  }
+
+  @Override
+  public void deleteHostAttributes() {
+    mapper.truncate();
+  }
+
+  @Override
+  public void saveHostAttributes(IHostAttributes hostAttributes) {
+    HostAttributes mutableAttributes = hostAttributes.newBuilder();
+
+    // 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());
+    if (existing.isPresent()) {
+      mutableAttributes.setMode(existing.get().getMode());
+    }
+
+    merge(IHostAttributes.build(mutableAttributes));
+  }
+
+  private static final Predicate<IAttribute> EMPTY_VALUES = new Predicate<IAttribute>() {
+    @Override
+    public boolean apply(IAttribute attribute) {
+      return attribute.getValues().isEmpty();
+    }
+  };
+
+  private void merge(IHostAttributes hostAttributes) {
+    if (Iterables.any(hostAttributes.getAttributes(), EMPTY_VALUES)) {
+      throw new IllegalArgumentException(
+          "Host attributes contains empty values: " + hostAttributes);
+    }
+
+    mapper.deleteAttributesAndValues(hostAttributes.getSlaveId());
+    mapper.insert(hostAttributes);
+    if (!hostAttributes.getAttributes().isEmpty()) {
+      mapper.insertAttributeValues(hostAttributes);
+    }
+  }
+
+  @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;
+    }
+  }
+
+  @Override
+  public Optional<IHostAttributes> getHostAttributes(String host) {
+    return Optional.fromNullable(mapper.select(host)).transform(IHostAttributes.FROM_BUILDER);
+  }
+
+  @Override
+  public Set<IHostAttributes> getHostAttributes() {
+    return IHostAttributes.setFromBuilders(mapper.selectAll());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/5cf760bf/src/main/java/org/apache/aurora/scheduler/storage/db/DbModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/DbModule.java b/src/main/java/org/apache/aurora/scheduler/storage/db/DbModule.java
index 99d725b..18925fd 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/db/DbModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/DbModule.java
@@ -23,9 +23,11 @@ import com.google.inject.PrivateModule;
 import com.google.inject.name.Names;
 import com.twitter.common.inject.Bindings;
 
+import org.apache.aurora.scheduler.storage.AttributeStore;
 import org.apache.aurora.scheduler.storage.LockStore;
 import org.apache.aurora.scheduler.storage.QuotaStore;
 import org.apache.aurora.scheduler.storage.Storage;
+import org.apache.aurora.scheduler.storage.db.typehandlers.TypeHandlers;
 import org.apache.ibatis.session.AutoMappingBehavior;
 import org.apache.ibatis.session.SqlSessionFactory;
 import org.apache.ibatis.transaction.jdbc.JdbcTransactionFactory;
@@ -102,6 +104,8 @@ public class DbModule extends PrivateModule {
 
         bindDataSourceProviderType(PooledDataSourceProvider.class);
         bindTransactionFactoryType(JdbcTransactionFactory.class);
+        addMapperClass(AttributeMapper.class);
+        addMapperClass(EnumValueMapper.class);
         addMapperClass(LockMapper.class);
         addMapperClass(JobKeyMapper.class);
         addMapperClass(QuotaMapper.class);
@@ -115,9 +119,12 @@ public class DbModule extends PrivateModule {
         // http://mybatis.github.io/mybatis-3/configuration.html#settings
         autoMappingBehavior(AutoMappingBehavior.FULL);
 
+        addTypeHandlersClasses(TypeHandlers.getAll());
+
         // TODO(davmclau): ensure that mybatis logging is configured correctly.
       }
     });
+    bindStore(AttributeStore.Mutable.class, DbAttributeStore.class);
     bindStore(LockStore.Mutable.class, DbLockStore.class);
     bindStore(QuotaStore.Mutable.class, DbQuotaStore.class);
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/5cf760bf/src/main/java/org/apache/aurora/scheduler/storage/db/DbStorage.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/DbStorage.java b/src/main/java/org/apache/aurora/scheduler/storage/db/DbStorage.java
index 0d8ad61..d3764c3 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/db/DbStorage.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/DbStorage.java
@@ -15,13 +15,13 @@ package org.apache.aurora.scheduler.storage.db;
 
 import java.io.IOException;
 import java.io.InputStreamReader;
-import java.util.Objects;
 
 import com.google.common.base.Charsets;
 import com.google.common.io.CharStreams;
 import com.google.common.util.concurrent.AbstractIdleService;
 import com.google.inject.Inject;
 
+import org.apache.aurora.gen.MaintenanceMode;
 import org.apache.aurora.scheduler.storage.AttributeStore;
 import org.apache.aurora.scheduler.storage.JobStore;
 import org.apache.aurora.scheduler.storage.LockStore;
@@ -31,6 +31,7 @@ import org.apache.aurora.scheduler.storage.Storage;
 import org.apache.aurora.scheduler.storage.TaskStore;
 import org.apache.ibatis.builder.StaticSqlSource;
 import org.apache.ibatis.exceptions.PersistenceException;
+import org.apache.ibatis.logging.jdk14.Jdk14LoggingImpl;
 import org.apache.ibatis.mapping.MappedStatement.Builder;
 import org.apache.ibatis.mapping.SqlCommandType;
 import org.apache.ibatis.session.Configuration;
@@ -38,6 +39,8 @@ import org.apache.ibatis.session.SqlSession;
 import org.apache.ibatis.session.SqlSessionFactory;
 import org.mybatis.guice.transactional.Transactional;
 
+import static java.util.Objects.requireNonNull;
+
 /**
  * A storage implementation backed by a relational database.
  *
@@ -53,14 +56,21 @@ class DbStorage extends AbstractIdleService implements Storage {
 
   private final SqlSessionFactory sessionFactory;
   private final MutableStoreProvider storeProvider;
+  private final EnumValueMapper enumValueMapper;
 
   @Inject
   DbStorage(
-      final SqlSessionFactory sessionFactory,
+      SqlSessionFactory sessionFactory,
+      EnumValueMapper enumValueMapper,
+      final AttributeStore.Mutable attributeStore,
       final LockStore.Mutable lockStore,
       final QuotaStore.Mutable quotaStore) {
 
-    this.sessionFactory = Objects.requireNonNull(sessionFactory);
+    this.sessionFactory = requireNonNull(sessionFactory);
+    this.enumValueMapper = requireNonNull(enumValueMapper);
+    requireNonNull(attributeStore);
+    requireNonNull(lockStore);
+    requireNonNull(quotaStore);
     storeProvider = new MutableStoreProvider() {
       @Override
       public SchedulerStore.Mutable getSchedulerStore() {
@@ -94,7 +104,7 @@ class DbStorage extends AbstractIdleService implements Storage {
 
       @Override
       public AttributeStore.Mutable getAttributeStore() {
-        throw new UnsupportedOperationException("Not yet implemented.");
+        return attributeStore;
       }
     };
   }
@@ -158,9 +168,15 @@ class DbStorage extends AbstractIdleService implements Storage {
         SqlCommandType.UPDATE)
         .build());
 
+    configuration.setLogImpl(Jdk14LoggingImpl.class);
+
     try (SqlSession session = sessionFactory.openSession()) {
       session.update(createStatementName);
     }
+
+    for (MaintenanceMode mode : MaintenanceMode.values()) {
+      enumValueMapper.addEnumValue("maintenance_modes", mode.getValue(), mode.name());
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/5cf760bf/src/main/java/org/apache/aurora/scheduler/storage/db/EnumValueMapper.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/EnumValueMapper.java b/src/main/java/org/apache/aurora/scheduler/storage/db/EnumValueMapper.java
new file mode 100644
index 0000000..33948a7
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/EnumValueMapper.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.aurora.scheduler.storage.db;
+
+import org.apache.ibatis.annotations.Param;
+
+/**
+ * Mapper used to populate static enum value tables.
+ */
+interface EnumValueMapper {
+
+  /**
+   * Inserts an enum value.
+   *
+   * @param table Name of the enum value table.
+   * @param id Unique enum ID.
+   * @param name Human-readable enum name.
+   */
+  void addEnumValue(@Param("table") String table, @Param("id") int id, @Param("name") String name);
+}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/5cf760bf/src/main/java/org/apache/aurora/scheduler/storage/db/MigrationModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/MigrationModule.java b/src/main/java/org/apache/aurora/scheduler/storage/db/MigrationModule.java
index 77cd8ab..ace97e3 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/db/MigrationModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/MigrationModule.java
@@ -16,6 +16,7 @@ package org.apache.aurora.scheduler.storage.db;
 import com.google.inject.AbstractModule;
 import com.twitter.common.inject.Bindings.KeyFactory;
 
+import org.apache.aurora.scheduler.storage.AttributeStore;
 import org.apache.aurora.scheduler.storage.LockStore;
 import org.apache.aurora.scheduler.storage.QuotaStore;
 
@@ -48,6 +49,7 @@ public class MigrationModule extends AbstractModule {
 
   @Override
   protected void configure() {
+    link(AttributeStore.Mutable.class);
     link(LockStore.Mutable.class);
     link(QuotaStore.Mutable.class);
   }

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/5cf760bf/src/main/java/org/apache/aurora/scheduler/storage/db/typehandlers/AbstractTEnumTypeHandler.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/typehandlers/AbstractTEnumTypeHandler.java b/src/main/java/org/apache/aurora/scheduler/storage/db/typehandlers/AbstractTEnumTypeHandler.java
new file mode 100644
index 0000000..892af5d
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/typehandlers/AbstractTEnumTypeHandler.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.aurora.scheduler.storage.db.typehandlers;
+
+import java.sql.CallableStatement;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+
+import org.apache.ibatis.type.JdbcType;
+import org.apache.ibatis.type.TypeHandler;
+import org.apache.thrift.TEnum;
+
+/**
+ * Type handler for fields of type {@link TEnum}.  Implementers need only override
+ * {@link #fromValue(int)}.
+ *
+ * @param <T> Enum type.
+ */
+abstract class AbstractTEnumTypeHandler<T extends TEnum> implements TypeHandler<T> {
+
+  /**
+   * Finds the enum value associated with the provided integer identity.
+   *
+   * @param value Value to find in the enum values.
+   * @return Enum value associated with {@code value}.
+   */
+  protected abstract T fromValue(int value);
+
+  @Override
+  public final void setParameter(PreparedStatement ps, int i, T parameter, JdbcType jdbcType)
+      throws SQLException {
+
+    ps.setInt(i, parameter.getValue());
+  }
+
+  @Override
+  public final T getResult(ResultSet rs, String columnName) throws SQLException {
+    return fromValue(rs.getInt(columnName));
+  }
+
+  @Override
+  public final T getResult(ResultSet rs, int columnIndex) throws SQLException {
+    return fromValue(rs.getInt(columnIndex));
+  }
+
+  @Override
+  public final T getResult(CallableStatement cs, int columnIndex) throws SQLException {
+    return fromValue(cs.getInt(columnIndex));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/5cf760bf/src/main/java/org/apache/aurora/scheduler/storage/db/typehandlers/MaintenanceModeTypeHandler.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/typehandlers/MaintenanceModeTypeHandler.java b/src/main/java/org/apache/aurora/scheduler/storage/db/typehandlers/MaintenanceModeTypeHandler.java
new file mode 100644
index 0000000..061027b
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/typehandlers/MaintenanceModeTypeHandler.java
@@ -0,0 +1,26 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.aurora.scheduler.storage.db.typehandlers;
+
+import org.apache.aurora.gen.MaintenanceMode;
+
+/**
+ * Type handler for {@link MaintenanceMode}.
+ */
+class MaintenanceModeTypeHandler extends AbstractTEnumTypeHandler<MaintenanceMode> {
+  @Override
+  protected MaintenanceMode fromValue(int value) {
+    return MaintenanceMode.findByValue(value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/5cf760bf/src/main/java/org/apache/aurora/scheduler/storage/db/typehandlers/TypeHandlers.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/typehandlers/TypeHandlers.java b/src/main/java/org/apache/aurora/scheduler/storage/db/typehandlers/TypeHandlers.java
new file mode 100644
index 0000000..f014123
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/typehandlers/TypeHandlers.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.aurora.scheduler.storage.db.typehandlers;
+
+import java.util.List;
+
+import com.google.common.collect.ImmutableList;
+
+import org.apache.ibatis.type.TypeHandler;
+
+/**
+ * Utility class to access the available type handler classes.
+ */
+public final class TypeHandlers {
+  private TypeHandlers() {
+    // Utility class.
+  }
+
+  public static List<Class<? extends TypeHandler<?>>> getAll() {
+    return ImmutableList.<Class<? extends TypeHandler<?>>>of(MaintenanceModeTypeHandler.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/5cf760bf/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
deleted file mode 100644
index 82fcddd..0000000
--- a/src/main/java/org/apache/aurora/scheduler/storage/mem/MemAttributeStore.java
+++ /dev/null
@@ -1,89 +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 java.util.Set;
-import java.util.concurrent.ConcurrentMap;
-
-import com.google.common.base.Optional;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Maps;
-
-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, IHostAttributes> hostAttributes = Maps.newConcurrentMap();
-
-  @Override
-  public void deleteHostAttributes() {
-    hostAttributes.clear();
-  }
-
-  @Override
-  public void saveHostAttributes(IHostAttributes attributes) {
-    hostAttributes.put(
-        attributes.getHost(),
-        merge(attributes, Optional.fromNullable(hostAttributes.get(attributes.getHost()))));
-  }
-
-  private IHostAttributes merge(IHostAttributes newAttributes, Optional<IHostAttributes> previous) {
-    HostAttributes attributes = newAttributes.newBuilder();
-    if (!attributes.isSetMode()) {
-      // If the newly-saved value does not explicitly set the mode, use the previous value
-      // or the default.
-      MaintenanceMode mode;
-      if (previous.isPresent() && previous.get().isSetMode()) {
-        mode = previous.get().getMode();
-      } else {
-        mode = MaintenanceMode.NONE;
-      }
-      attributes.setMode(mode);
-    }
-    if (!attributes.isSetAttributes()) {
-      attributes.setAttributes(ImmutableSet.<Attribute>of());
-    }
-    return IHostAttributes.build(attributes);
-  }
-
-  @Override
-  public boolean setMaintenanceMode(String host, MaintenanceMode mode) {
-    IHostAttributes stored = hostAttributes.get(host);
-    if (stored == null) {
-      return false;
-    } else {
-      hostAttributes.replace(
-          host,
-          stored,
-          IHostAttributes.build(stored.newBuilder().setMode(mode)));
-      return true;
-    }
-  }
-
-  @Override
-  public Optional<IHostAttributes> getHostAttributes(String host) {
-    return Optional.fromNullable(hostAttributes.get(host));
-  }
-
-  @Override
-  public Set<IHostAttributes> getHostAttributes() {
-    return ImmutableSet.copyOf(hostAttributes.values());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/5cf760bf/src/main/java/org/apache/aurora/scheduler/storage/mem/MemStorage.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/mem/MemStorage.java b/src/main/java/org/apache/aurora/scheduler/storage/mem/MemStorage.java
index 5cddd12..1e082ac 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/mem/MemStorage.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/mem/MemStorage.java
@@ -86,7 +86,7 @@ public class MemStorage implements Storage {
       @Delegated final LockStore.Mutable lockStore,
       @Delegated final Storage delegated,
       @Delegated final QuotaStore.Mutable quotaStore,
-      final AttributeStore.Mutable attributeStore) {
+      @Delegated final AttributeStore.Mutable attributeStore) {
     this.delegatedStore = delegated;
     storeProvider = new MutableStoreProvider() {
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/5cf760bf/src/main/java/org/apache/aurora/scheduler/storage/mem/MemStorageModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/mem/MemStorageModule.java b/src/main/java/org/apache/aurora/scheduler/storage/mem/MemStorageModule.java
index 65565f1..4c9d69c 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/mem/MemStorageModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/mem/MemStorageModule.java
@@ -19,7 +19,6 @@ import com.google.inject.Key;
 import com.google.inject.PrivateModule;
 import com.twitter.common.inject.Bindings.KeyFactory;
 
-import org.apache.aurora.scheduler.storage.AttributeStore;
 import org.apache.aurora.scheduler.storage.JobStore;
 import org.apache.aurora.scheduler.storage.SchedulerStore;
 import org.apache.aurora.scheduler.storage.Storage;
@@ -74,6 +73,5 @@ public final class MemStorageModule extends PrivateModule {
     bindStore(SchedulerStore.Mutable.class, MemSchedulerStore.class);
     bindStore(JobStore.Mutable.class, MemJobStore.class);
     bindStore(TaskStore.Mutable.class, MemTaskStore.class);
-    bindStore(AttributeStore.Mutable.class, MemAttributeStore.class);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/5cf760bf/src/main/resources/org/apache/aurora/scheduler/storage/db/AttributeMapper.xml
----------------------------------------------------------------------
diff --git a/src/main/resources/org/apache/aurora/scheduler/storage/db/AttributeMapper.xml b/src/main/resources/org/apache/aurora/scheduler/storage/db/AttributeMapper.xml
new file mode 100644
index 0000000..f9b689d
--- /dev/null
+++ b/src/main/resources/org/apache/aurora/scheduler/storage/db/AttributeMapper.xml
@@ -0,0 +1,81 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!DOCTYPE mapper
+    PUBLIC "-//mybatis.org//DTD Mapper 3.0//EN"
+    "http://mybatis.org/dtd/mybatis-3-mapper.dtd">
+<mapper namespace="org.apache.aurora.scheduler.storage.db.AttributeMapper">
+  <insert id="insert">
+    INSERT INTO host_attributes (
+      host,
+      mode,
+      slave_id
+    ) VALUES (
+      #{host},
+      #{mode, typeHandler=org.apache.aurora.scheduler.storage.db.typehandlers.MaintenanceModeTypeHandler},
+      #{slaveId}
+    )
+  </insert>
+
+  <insert id="insertAttributeValues">
+    INSERT INTO host_attribute_values (
+      host_attribute_id,
+      name,
+      value
+    ) VALUES
+    <foreach item="attribute" collection="attributes" separator=",">
+      <foreach item="value" collection="attribute.values" open="(" separator="),(" close=")">
+        (SELECT id FROM host_attributes WHERE slave_id = #{slaveId}),
+        #{attribute.name},
+        #{value}
+      </foreach>
+    </foreach>
+  </insert>
+
+  <delete id="deleteAttributesAndValues">
+    <!-- This assumes the schema enables cascading deletes in the values table. -->
+    DELETE FROM host_attributes
+    WHERE slave_id = #{id}
+  </delete>
+
+  <resultMap id="hostAttributeResultMap" type="org.apache.aurora.gen.HostAttributes">
+    <id column="a_id" />
+    <result property="mode"
+            column="a_mode"
+            typeHandler="org.apache.aurora.scheduler.storage.db.typehandlers.MaintenanceModeTypeHandler" />
+    <result property="host" column="a_host" />
+    <result property="slaveId" column="a_slave_id" />
+    <collection property="attributes" ofType="org.apache.aurora.gen.Attribute" columnPrefix="v_">
+      <id column="name" property="name" />
+      <collection property="values" ofType="String">
+        <result column="value" />
+      </collection>
+    </collection>
+  </resultMap>
+
+  <sql id="unscoped_select">
+    SELECT
+      a.id AS a_id,
+      a.host AS a_host,
+      a.mode AS a_mode,
+      a.slave_id AS a_slave_id,
+      a.slave_id AS v_slave_id,
+      v.id AS v_id,
+      v.name AS v_name,
+      v.value AS v_value
+    FROM host_attributes as a
+    LEFT OUTER JOIN host_attribute_values AS v ON v.host_attribute_id = a.id
+  </sql>
+
+  <select id="select" resultMap="hostAttributeResultMap">
+    <include refid="unscoped_select"/>
+    WHERE host = #{id}
+  </select>
+
+  <select id="selectAll" resultMap="hostAttributeResultMap">
+    <include refid="unscoped_select"/>
+  </select>
+
+  <delete id="truncate">
+    DELETE FROM host_attribute_values;
+    DELETE FROM host_attributes
+  </delete>
+</mapper>

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/5cf760bf/src/main/resources/org/apache/aurora/scheduler/storage/db/EnumValueMapper.xml
----------------------------------------------------------------------
diff --git a/src/main/resources/org/apache/aurora/scheduler/storage/db/EnumValueMapper.xml b/src/main/resources/org/apache/aurora/scheduler/storage/db/EnumValueMapper.xml
new file mode 100644
index 0000000..153fd26
--- /dev/null
+++ b/src/main/resources/org/apache/aurora/scheduler/storage/db/EnumValueMapper.xml
@@ -0,0 +1,15 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!DOCTYPE mapper
+    PUBLIC "-//mybatis.org//DTD Mapper 3.0//EN"
+    "http://mybatis.org/dtd/mybatis-3-mapper.dtd">
+<mapper namespace="org.apache.aurora.scheduler.storage.db.EnumValueMapper">
+  <insert id="addEnumValue">
+    INSERT INTO ${table} (
+      id,
+      name
+    ) VALUES (
+      #{id},
+      #{name}
+    )
+  </insert>
+</mapper>

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/5cf760bf/src/main/resources/org/apache/aurora/scheduler/storage/db/LockMapper.xml
----------------------------------------------------------------------
diff --git a/src/main/resources/org/apache/aurora/scheduler/storage/db/LockMapper.xml b/src/main/resources/org/apache/aurora/scheduler/storage/db/LockMapper.xml
index 31b98cb..9e1f8e6 100644
--- a/src/main/resources/org/apache/aurora/scheduler/storage/db/LockMapper.xml
+++ b/src/main/resources/org/apache/aurora/scheduler/storage/db/LockMapper.xml
@@ -60,7 +60,7 @@
     JOIN job_keys AS key ON job_key_id = key.id
   </select>
   <sql id="jobKeyScope">
-    OUTER JOIN job_keys AS key ON key.role = #{job.role}
+    LEFT OUTER JOIN job_keys AS key ON key.role = #{job.role}
     AND key.environment = #{job.environment}
     AND key.name = #{job.name}
     AND key.id = job_key_id

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/5cf760bf/src/main/resources/org/apache/aurora/scheduler/storage/db/schema.sql
----------------------------------------------------------------------
diff --git a/src/main/resources/org/apache/aurora/scheduler/storage/db/schema.sql b/src/main/resources/org/apache/aurora/scheduler/storage/db/schema.sql
index 3298eb3..0bb6cd2 100644
--- a/src/main/resources/org/apache/aurora/scheduler/storage/db/schema.sql
+++ b/src/main/resources/org/apache/aurora/scheduler/storage/db/schema.sql
@@ -43,3 +43,30 @@ CREATE TABLE quotas(
 
   UNIQUE(role)
 );
+
+CREATE TABLE maintenance_modes(
+  id INT PRIMARY KEY,
+  name VARCHAR NOT NULL,
+
+  UNIQUE(name)
+);
+
+CREATE TABLE host_attributes(
+  id INT IDENTITY,
+  host VARCHAR NOT NULL,
+  mode TINYINT NOT NULL REFERENCES maintenance_modes(id),
+  slave_id VARCHAR NOT NULL,
+
+  UNIQUE(host),
+  UNIQUE(slave_id),
+);
+
+CREATE TABLE host_attribute_values(
+  id INT IDENTITY,
+  host_attribute_id INT NOT NULL REFERENCES host_attributes(id)
+  ON DELETE CASCADE,
+  name VARCHAR NOT NULL,
+  value VARCHAR NOT NULL,
+
+  UNIQUE(host_attribute_id, name, value)
+);

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/5cf760bf/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
new file mode 100644
index 0000000..3962fb0
--- /dev/null
+++ b/src/test/java/org/apache/aurora/scheduler/storage/db/DbAttributeStoreTest.java
@@ -0,0 +1,184 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.aurora.scheduler.storage.db;
+
+import java.io.IOException;
+import java.util.Set;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableSet;
+import com.google.inject.Guice;
+import com.google.inject.Injector;
+import com.twitter.common.inject.Bindings;
+
+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.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 org.junit.Before;
+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 {
+
+  private static final String HOST_A = "hostA";
+  private static final String HOST_B = "hostB";
+  private static final String SLAVE_A = "slaveA";
+  private static final String SLAVE_B = "slaveB";
+  private static final Attribute ATTR1 = new Attribute("attr1", ImmutableSet.of("a", "b", "c"));
+  private static final Attribute ATTR2 = new Attribute("attr2", ImmutableSet.of("d", "e", "f"));
+  private static final Attribute ATTR3 = new Attribute("attr3", ImmutableSet.of("a", "d", "g"));
+  private static final IHostAttributes HOST_A_ATTRS =
+      IHostAttributes.build(new HostAttributes(HOST_A, ImmutableSet.of(ATTR1, ATTR2))
+          .setSlaveId(SLAVE_A)
+          .setMode(MaintenanceMode.NONE));
+  private static final IHostAttributes HOST_B_ATTRS =
+      IHostAttributes.build(new HostAttributes(HOST_B, ImmutableSet.of(ATTR2, ATTR3))
+          .setSlaveId(SLAVE_B)
+          .setMode(MaintenanceMode.DRAINING));
+
+  private Storage storage;
+
+  @Before
+  public void setUp() throws IOException {
+    Injector injector = Guice.createInjector(DbModule.testModule(Bindings.KeyFactory.PLAIN));
+    storage = injector.getInstance(Storage.class);
+    storage.prepare();
+  }
+
+  @Test
+  public void testCrud() {
+    assertEquals(Optional.<IHostAttributes>absent(), read(HOST_A));
+    assertEquals(ImmutableSet.<IHostAttributes>of(), readAll());
+
+    insert(HOST_A_ATTRS);
+    assertEquals(Optional.of(HOST_A_ATTRS), read(HOST_A));
+    assertEquals(ImmutableSet.of(HOST_A_ATTRS), readAll());
+
+    insert(HOST_B_ATTRS);
+    assertEquals(Optional.of(HOST_B_ATTRS), read(HOST_B));
+    assertEquals(ImmutableSet.of(HOST_A_ATTRS, HOST_B_ATTRS), readAll());
+
+    IHostAttributes updatedA = IHostAttributes.build(
+        HOST_A_ATTRS.newBuilder().setAttributes(ImmutableSet.of(ATTR1, ATTR3)));
+    insert(updatedA);
+    assertEquals(Optional.of(updatedA), read(HOST_A));
+    assertEquals(ImmutableSet.of(updatedA, HOST_B_ATTRS), readAll());
+
+    truncate();
+    assertEquals(Optional.<IHostAttributes>absent(), read(HOST_A));
+    assertEquals(ImmutableSet.<IHostAttributes>of(), readAll());
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testEmptyAttributeValues() {
+    IHostAttributes attributes = IHostAttributes.build(HOST_A_ATTRS.newBuilder()
+        .setAttributes(ImmutableSet.of(new Attribute("attr1", ImmutableSet.<String>of()))));
+    insert(attributes);
+  }
+
+  @Test
+  public void testNoAttributes() {
+    IHostAttributes attributes = IHostAttributes.build(
+        HOST_A_ATTRS.newBuilder().setAttributes(ImmutableSet.<Attribute>of()));
+    insert(attributes);
+    assertEquals(Optional.of(attributes), read(HOST_A));
+  }
+
+  @Test
+  public void testSetMaintenanceMode() {
+    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
+  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));
+  }
+
+  private void insert(final IHostAttributes attributes) {
+    storage.write(new MutateWork.NoResult.Quiet() {
+      @Override
+      protected void execute(MutableStoreProvider storeProvider) {
+        storeProvider.getAttributeStore().saveHostAttributes(attributes);
+      }
+    });
+  }
+
+  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
+      public Optional<IHostAttributes> apply(StoreProvider storeProvider) {
+        return storeProvider.getAttributeStore().getHostAttributes(host);
+      }
+    });
+  }
+
+  private Set<IHostAttributes> readAll() {
+    return storage.consistentRead(new Work.Quiet<Set<IHostAttributes>>() {
+      @Override
+      public Set<IHostAttributes> apply(StoreProvider storeProvider) {
+        return storeProvider.getAttributeStore().getHostAttributes();
+      }
+    });
+  }
+
+  private void truncate() {
+    storage.write(new MutateWork.NoResult.Quiet() {
+      @Override
+      protected void execute(MutableStoreProvider storeProvider) {
+        storeProvider.getAttributeStore().deleteHostAttributes();
+      }
+    });
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/5cf760bf/src/test/java/org/apache/aurora/scheduler/storage/mem/MemAttributeStoreTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/storage/mem/MemAttributeStoreTest.java b/src/test/java/org/apache/aurora/scheduler/storage/mem/MemAttributeStoreTest.java
deleted file mode 100644
index 45fa43c..0000000
--- a/src/test/java/org/apache/aurora/scheduler/storage/mem/MemAttributeStoreTest.java
+++ /dev/null
@@ -1,67 +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 com.google.common.base.Optional;
-import com.google.common.collect.ImmutableSet;
-
-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.IHostAttributes;
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-
-public class MemAttributeStoreTest {
-
-  private static final IHostAttributes ATTRS = IHostAttributes.build(
-      new HostAttributes()
-          .setHost("hostA")
-          .setSlaveId("slaveA")
-          .setAttributes(ImmutableSet.of(
-              makeAttribute("host", "hostA"),
-              makeAttribute("rack", "rackA")))
-  );
-
-  private AttributeStore.Mutable store;
-
-  @Before
-  public void setUp() {
-    store = new MemAttributeStore();
-  }
-
-  @Test
-  public void testAttributeChange() {
-    store.saveHostAttributes(ATTRS);
-    assertEquals(Optional.of(defaultMode(ATTRS)), store.getHostAttributes(ATTRS.getHost()));
-    HostAttributes builder = ATTRS.newBuilder();
-    builder.addToAttributes(makeAttribute("foo", "bar"));
-    IHostAttributes updated = IHostAttributes.build(builder);
-    store.saveHostAttributes(updated);
-    assertEquals(Optional.of(defaultMode(updated)), store.getHostAttributes(ATTRS.getHost()));
-  }
-
-  private static Attribute makeAttribute(String name, String... values) {
-    return new Attribute()
-        .setName(name)
-        .setValues(ImmutableSet.<String>builder().add(values).build());
-  }
-
-  private static IHostAttributes defaultMode(IHostAttributes attrs) {
-    return IHostAttributes.build(attrs.newBuilder().setMode(MaintenanceMode.NONE));
-  }
-}