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 01:33:36 UTC

git commit: Database-backed implementation of QuotaStore.

Repository: incubator-aurora
Updated Branches:
  refs/heads/master 4be190f60 -> d8d056512


Database-backed implementation of QuotaStore.

Bugs closed: AURORA-421

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


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

Branch: refs/heads/master
Commit: d8d05651266288de1a5958f15a5bd8a9fe2dc768
Parents: 4be190f
Author: Bill Farner <wf...@apache.org>
Authored: Wed Jun 25 16:24:37 2014 -0700
Committer: Bill Farner <wf...@apache.org>
Committed: Wed Jun 25 16:24:37 2014 -0700

----------------------------------------------------------------------
 .../aurora/scheduler/storage/db/DbModule.java   |   4 +
 .../scheduler/storage/db/DbQuotaStore.java      |  69 +++++++++
 .../aurora/scheduler/storage/db/DbStorage.java  |  14 +-
 .../scheduler/storage/db/MigrationModule.java   |   8 +-
 .../scheduler/storage/db/QuotaMapper.java       |  63 ++++++++
 .../scheduler/storage/mem/MemQuotaStore.java    |  64 ---------
 .../scheduler/storage/mem/MemStorage.java       |   2 +-
 .../scheduler/storage/mem/MemStorageModule.java |   2 -
 .../aurora/scheduler/storage/db/QuotaMapper.xml |  44 ++++++
 .../aurora/scheduler/storage/db/schema.sql      |  10 ++
 .../scheduler/storage/db/DbQuotaStoreTest.java  | 142 +++++++++++++++++++
 11 files changed, 351 insertions(+), 71 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d8d05651/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 e365cd6..1738b95 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,6 +23,7 @@ import com.google.inject.name.Names;
 import com.twitter.common.inject.Bindings;
 
 import org.apache.aurora.scheduler.storage.LockStore;
+import org.apache.aurora.scheduler.storage.QuotaStore;
 import org.apache.aurora.scheduler.storage.Storage;
 import org.apache.ibatis.session.AutoMappingBehavior;
 import org.apache.ibatis.session.SqlSessionFactory;
@@ -44,6 +45,7 @@ import static com.google.inject.name.Names.named;
  *     <li>Keys provided by the provided{@code keyFactory} for:
  *        <ul>
  *          <li>{@link org.apache.aurora.scheduler.storage.LockStore.Mutable}</li>
+ *          <li>{@link org.apache.aurora.scheduler.storage.QuotaStore.Mutable}</li>
  *        </ul>
  *     </li>
  *   </ul>
@@ -81,6 +83,7 @@ public class DbModule extends PrivateModule {
         bindTransactionFactoryType(JdbcTransactionFactory.class);
         addMapperClass(LockMapper.class);
         addMapperClass(JobKeyMapper.class);
+        addMapperClass(QuotaMapper.class);
         Properties props = new Properties();
         // We have no plans to take advantage of multiple DB environments. This is a required
         // property though, so we use an unnamed environment.
@@ -95,6 +98,7 @@ public class DbModule extends PrivateModule {
       }
     });
     bindStore(LockStore.Mutable.class, DbLockStore.class);
+    bindStore(QuotaStore.Mutable.class, DbQuotaStore.class);
 
     Key<Storage> storageKey = keyFactory.create(Storage.class);
     bind(storageKey).to(DbStorage.class);

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d8d05651/src/main/java/org/apache/aurora/scheduler/storage/db/DbQuotaStore.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/DbQuotaStore.java b/src/main/java/org/apache/aurora/scheduler/storage/db/DbQuotaStore.java
new file mode 100644
index 0000000..0c327de
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/DbQuotaStore.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.Map;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableMap;
+import com.google.inject.Inject;
+
+import org.apache.aurora.gen.storage.SaveQuota;
+import org.apache.aurora.scheduler.storage.QuotaStore;
+import org.apache.aurora.scheduler.storage.entities.IResourceAggregate;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * Quota store backed by a relational database.
+ */
+class DbQuotaStore implements QuotaStore.Mutable {
+
+  private final QuotaMapper mapper;
+
+  @Inject
+  DbQuotaStore(QuotaMapper mapper) {
+    this.mapper = checkNotNull(mapper);
+  }
+
+  @Override
+  public Optional<IResourceAggregate> fetchQuota(String role) {
+    return Optional.fromNullable(mapper.select(role))
+        .transform(IResourceAggregate.FROM_BUILDER);
+  }
+
+  @Override
+  public Map<String, IResourceAggregate> fetchQuotas() {
+    ImmutableMap.Builder<String, IResourceAggregate> results = ImmutableMap.builder();
+    for (SaveQuota result : mapper.selectAll()) {
+      results.put(result.getRole(), IResourceAggregate.build(result.getQuota()));
+    }
+    return results.build();
+  }
+
+  @Override
+  public void deleteQuotas() {
+    mapper.truncate();
+  }
+
+  @Override
+  public void removeQuota(String role) {
+    mapper.delete(role);
+  }
+
+  @Override
+  public void saveQuota(String role, IResourceAggregate quota) {
+    mapper.merge(role, quota.newBuilder());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d8d05651/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 41755c3..c683e39 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
@@ -16,6 +16,7 @@ package org.apache.aurora.scheduler.storage.db;
 import java.io.IOException;
 import java.io.InputStreamReader;
 
+import com.google.common.base.Charsets;
 import com.google.common.base.Preconditions;
 import com.google.common.io.CharStreams;
 import com.google.common.util.concurrent.AbstractIdleService;
@@ -54,7 +55,11 @@ class DbStorage extends AbstractIdleService implements Storage {
   private final MutableStoreProvider storeProvider;
 
   @Inject
-  DbStorage(final SqlSessionFactory sessionFactory, final LockStore.Mutable lockStore) {
+  DbStorage(
+      final SqlSessionFactory sessionFactory,
+      final LockStore.Mutable lockStore,
+      final QuotaStore.Mutable quotaStore) {
+
     this.sessionFactory = Preconditions.checkNotNull(sessionFactory);
     storeProvider = new MutableStoreProvider() {
       @Override
@@ -84,7 +89,7 @@ class DbStorage extends AbstractIdleService implements Storage {
 
       @Override
       public QuotaStore.Mutable getQuotaStore() {
-        throw new UnsupportedOperationException("Not yet implemented.");
+        return quotaStore;
       }
 
       @Override
@@ -140,13 +145,16 @@ class DbStorage extends AbstractIdleService implements Storage {
   protected void startUp() throws IOException {
     Configuration configuration = sessionFactory.getConfiguration();
     String createStatementName = "create_tables";
+    configuration.setMapUnderscoreToCamelCase(true);
     configuration.addMappedStatement(new Builder(
         configuration,
         createStatementName,
         new StaticSqlSource(
             configuration,
             CharStreams.toString(
-                new InputStreamReader(DbStorage.class.getResourceAsStream("schema.sql"), "UTF-8"))),
+                new InputStreamReader(
+                    DbStorage.class.getResourceAsStream("schema.sql"),
+                    Charsets.UTF_8))),
         SqlCommandType.UPDATE)
         .build());
 

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d8d05651/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 7e98ebf..505c94d 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/db/MigrationModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/MigrationModule.java
@@ -17,6 +17,7 @@ import com.google.inject.AbstractModule;
 import com.twitter.common.inject.Bindings.KeyFactory;
 
 import org.apache.aurora.scheduler.storage.LockStore;
+import org.apache.aurora.scheduler.storage.QuotaStore;
 
 import static com.google.common.base.Preconditions.checkNotNull;
 
@@ -41,8 +42,13 @@ public class MigrationModule extends AbstractModule {
     this.toFactory = checkNotNull(to);
   }
 
+  private <T> void link(Class<T> clazz) {
+    bind(fromFactory.create(clazz)).to(toFactory.create(clazz));
+  }
+
   @Override
   protected void configure() {
-    bind(fromFactory.create(LockStore.Mutable.class)).to(toFactory.create(LockStore.Mutable.class));
+    link(LockStore.Mutable.class);
+    link(QuotaStore.Mutable.class);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d8d05651/src/main/java/org/apache/aurora/scheduler/storage/db/QuotaMapper.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/QuotaMapper.java b/src/main/java/org/apache/aurora/scheduler/storage/db/QuotaMapper.java
new file mode 100644
index 0000000..3e5c434
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/QuotaMapper.java
@@ -0,0 +1,63 @@
+/**
+ * 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.ResourceAggregate;
+import org.apache.aurora.gen.storage.SaveQuota;
+import org.apache.ibatis.annotations.Param;
+
+/**
+ * MyBatis mapper class for QuotaMapper.xml.
+ */
+interface QuotaMapper {
+  /**
+   * Saves the quota for the given {@code role}, updating the existing value if it exists.
+   *
+   * @param role Role to save quota for.
+   * @param quota Quota value to store.
+   */
+  void merge(@Param("role") String role, @Param("quota") ResourceAggregate quota);
+
+  /**
+   * Gets the quota assigned to a role.
+   *
+   * @param role Role to select quota for.
+   * @return The previously-saved quota for the role, if it exists.
+   */
+  @Nullable
+  ResourceAggregate select(String role);
+
+  /**
+   * Gets all saved quotas.
+   *
+   * @return All quotas stored in the database.
+   */
+  List<SaveQuota> selectAll();
+
+  /**
+   * Removes the quota stored for a role.
+   *
+   * @param role Role to delete the quota entry for, if one exists.
+   */
+  void delete(String role);
+
+  /**
+   * Removes all stored quota records.
+   */
+  void truncate();
+}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d8d05651/src/main/java/org/apache/aurora/scheduler/storage/mem/MemQuotaStore.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/mem/MemQuotaStore.java b/src/main/java/org/apache/aurora/scheduler/storage/mem/MemQuotaStore.java
deleted file mode 100644
index 49a1ba7..0000000
--- a/src/main/java/org/apache/aurora/scheduler/storage/mem/MemQuotaStore.java
+++ /dev/null
@@ -1,64 +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.Map;
-
-import com.google.common.base.Optional;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Maps;
-
-import org.apache.aurora.scheduler.storage.QuotaStore;
-import org.apache.aurora.scheduler.storage.entities.IResourceAggregate;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * An in-memory quota store.
- */
-class MemQuotaStore implements QuotaStore.Mutable {
-
-  private final Map<String, IResourceAggregate> quotas = Maps.newConcurrentMap();
-
-  @Override
-  public void deleteQuotas() {
-    quotas.clear();
-  }
-
-  @Override
-  public void removeQuota(String role) {
-    checkNotNull(role);
-
-    quotas.remove(role);
-  }
-
-  @Override
-  public void saveQuota(String role, IResourceAggregate quota) {
-    checkNotNull(role);
-    checkNotNull(quota);
-
-    quotas.put(role, quota);
-  }
-
-  @Override
-  public Optional<IResourceAggregate> fetchQuota(String role) {
-    checkNotNull(role);
-    return Optional.fromNullable(quotas.get(role));
-  }
-
-  @Override
-  public Map<String, IResourceAggregate> fetchQuotas() {
-    return ImmutableMap.copyOf(quotas);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d8d05651/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 90d9a65..65750b6 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
@@ -85,7 +85,7 @@ public class MemStorage implements Storage {
       final TaskStore.Mutable taskStore,
       @Delegated final LockStore.Mutable lockStore,
       @Delegated final Storage delegated,
-      final QuotaStore.Mutable quotaStore,
+      @Delegated final QuotaStore.Mutable quotaStore,
       final AttributeStore.Mutable attributeStore) {
     this.delegatedStore = delegated;
     storeProvider = new MutableStoreProvider() {

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d8d05651/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 acafd40..23555c2 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
@@ -21,7 +21,6 @@ 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.QuotaStore;
 import org.apache.aurora.scheduler.storage.SchedulerStore;
 import org.apache.aurora.scheduler.storage.Storage;
 import org.apache.aurora.scheduler.storage.Storage.Volatile;
@@ -75,7 +74,6 @@ 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(QuotaStore.Mutable.class, MemQuotaStore.class);
     bindStore(AttributeStore.Mutable.class, MemAttributeStore.class);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d8d05651/src/main/resources/org/apache/aurora/scheduler/storage/db/QuotaMapper.xml
----------------------------------------------------------------------
diff --git a/src/main/resources/org/apache/aurora/scheduler/storage/db/QuotaMapper.xml b/src/main/resources/org/apache/aurora/scheduler/storage/db/QuotaMapper.xml
new file mode 100644
index 0000000..aff4b63
--- /dev/null
+++ b/src/main/resources/org/apache/aurora/scheduler/storage/db/QuotaMapper.xml
@@ -0,0 +1,44 @@
+<?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.QuotaMapper">
+  <insert id="merge">
+    MERGE INTO quotas (
+      id,
+      role,
+      num_cpus,
+      ram_mb,
+      disk_mb
+    ) KEY(role) VALUES (
+      null,
+      #{role},
+      #{quota.numCpus},
+      #{quota.ramMb},
+      #{quota.diskMb}
+    )
+  </insert>
+
+  <select id="select" resultType="org.apache.aurora.gen.ResourceAggregate">
+    SELECT * FROM quotas
+    WHERE role = #{id}
+  </select>
+
+  <resultMap id="quotaResultMap" type="org.apache.aurora.gen.storage.SaveQuota">
+    <id column="id" />
+    <association property="quota" javaType="org.apache.aurora.gen.ResourceAggregate" />
+  </resultMap>
+
+  <select id="selectAll" resultMap="quotaResultMap">
+    SELECT * FROM quotas
+  </select>
+
+  <delete id="delete">
+    DELETE FROM quotas
+    WHERE role = #{id}
+  </delete>
+
+  <delete id="truncate">
+    DELETE FROM quotas
+  </delete>
+</mapper>

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d8d05651/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 405fda5..35a72ab 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
@@ -19,3 +19,13 @@ CREATE TABLE locks(
 
   UNIQUE(job_key_id)
 );
+
+CREATE TABLE quotas(
+  id INT IDENTITY,
+  role VARCHAR NOT NULL,
+  num_cpus FLOAT NOT NULL,
+  ram_mb INT NOT NULL,
+  disk_mb INT NOT NULL,
+
+  UNIQUE(role)
+);

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d8d05651/src/test/java/org/apache/aurora/scheduler/storage/db/DbQuotaStoreTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/storage/db/DbQuotaStoreTest.java b/src/test/java/org/apache/aurora/scheduler/storage/db/DbQuotaStoreTest.java
new file mode 100644
index 0000000..21151f5
--- /dev/null
+++ b/src/test/java/org/apache/aurora/scheduler/storage/db/DbQuotaStoreTest.java
@@ -0,0 +1,142 @@
+/**
+ * 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.Map;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableMap;
+import com.google.inject.Guice;
+import com.google.inject.Injector;
+import com.twitter.common.inject.Bindings;
+
+import org.apache.aurora.gen.ResourceAggregate;
+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.IResourceAggregate;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class DbQuotaStoreTest {
+
+  private static final String ROLE_A = "roleA";
+  private static final String ROLE_B = "roleB";
+  private static final IResourceAggregate QUOTA_A =
+      IResourceAggregate.build(new ResourceAggregate(1.0D, 2, 3));
+  private static final IResourceAggregate QUOTA_B =
+      IResourceAggregate.build(new ResourceAggregate(2.0D, 4, 6));
+
+  private Storage storage;
+
+  @Before
+  public void setUp() throws IOException {
+    Injector injector = Guice.createInjector(new DbModule(Bindings.KeyFactory.PLAIN));
+    storage = injector.getInstance(Storage.class);
+    storage.prepare();
+  }
+
+  @Test
+  public void testCrud() {
+    assertEquals(Optional.<IResourceAggregate>absent(), select(ROLE_A));
+    assertQuotas(ImmutableMap.<String, IResourceAggregate>of());
+
+    save(ROLE_A, QUOTA_A);
+    save(ROLE_B, QUOTA_B);
+
+    assertEquals(Optional.of(QUOTA_A), select(ROLE_A));
+    assertEquals(Optional.of(QUOTA_B), select(ROLE_B));
+    assertQuotas(ImmutableMap.of(ROLE_A, QUOTA_A, ROLE_B, QUOTA_B));
+
+    delete(ROLE_B);
+    assertEquals(Optional.of(QUOTA_A), select(ROLE_A));
+    assertEquals(Optional.<IResourceAggregate>absent(), select(ROLE_B));
+    assertQuotas(ImmutableMap.of(ROLE_A, QUOTA_A));
+
+    deleteAll();
+    assertEquals(Optional.<IResourceAggregate>absent(), select(ROLE_A));
+    assertEquals(Optional.<IResourceAggregate>absent(), select(ROLE_B));
+    assertQuotas(ImmutableMap.<String, IResourceAggregate>of());
+  }
+
+  @Test
+  public void testDeleteNonExistent() {
+    assertEquals(Optional.<IResourceAggregate>absent(), select(ROLE_A));
+    assertQuotas(ImmutableMap.<String, IResourceAggregate>of());
+    delete(ROLE_A);
+    assertEquals(Optional.<IResourceAggregate>absent(), select(ROLE_A));
+    assertQuotas(ImmutableMap.<String, IResourceAggregate>of());
+  }
+
+  @Test
+  public void testUpsert() {
+    save(ROLE_A, QUOTA_A);
+    save(ROLE_A, QUOTA_B);
+    assertEquals(Optional.of(QUOTA_B), select(ROLE_A));
+    assertQuotas(ImmutableMap.of(ROLE_A, QUOTA_B));
+  }
+
+  private void save(final String role, final IResourceAggregate quota) {
+    storage.write(new MutateWork.NoResult.Quiet() {
+      @Override
+      public void execute(MutableStoreProvider storeProvider) {
+        storeProvider.getQuotaStore().saveQuota(role, quota);
+      }
+    });
+  }
+
+  private Optional<IResourceAggregate> select(final String role) {
+    return storage.consistentRead(new Work.Quiet<Optional<IResourceAggregate>>() {
+      @Override
+      public Optional<IResourceAggregate> apply(StoreProvider storeProvider) {
+        return storeProvider.getQuotaStore().fetchQuota(role);
+      }
+    });
+  }
+
+  private void assertQuotas(Map<String, IResourceAggregate> quotas) {
+    assertEquals(
+        quotas,
+        storage.consistentRead(new Work.Quiet<Map<String, IResourceAggregate>>() {
+          @Override
+          public Map<String, IResourceAggregate> apply(StoreProvider storeProvider) {
+            return storeProvider.getQuotaStore().fetchQuotas();
+          }
+        })
+    );
+  }
+
+  private void delete(final String role) {
+    storage.write(new MutateWork.NoResult.Quiet() {
+      @Override
+      public void execute(MutableStoreProvider storeProvider) {
+        storeProvider.getQuotaStore().removeQuota(role);
+      }
+    });
+  }
+
+  private void deleteAll() {
+    storage.write(new MutateWork.NoResult.Quiet() {
+      @Override
+      public void execute(MutableStoreProvider storeProvider) {
+        storeProvider.getQuotaStore().deleteQuotas();
+      }
+    });
+  }
+}