You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by ma...@apache.org on 2015/09/09 21:04:13 UTC

aurora git commit: Dropping quota check for dedicated jobs and exposing dedicated consumption.

Repository: aurora
Updated Branches:
  refs/heads/master 4577de4dd -> 01e611ad4


Dropping quota check for dedicated jobs and exposing dedicated consumption.

Bugs closed: AURORA-1460

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


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

Branch: refs/heads/master
Commit: 01e611ad476ef57e16d8996c7b47b7cec0726977
Parents: 4577de4
Author: Maxim Khutornenko <ma...@apache.org>
Authored: Wed Sep 9 12:03:27 2015 -0700
Committer: Maxim Khutornenko <ma...@apache.org>
Committed: Wed Sep 9 12:03:27 2015 -0700

----------------------------------------------------------------------
 NEWS                                            |   1 +
 .../thrift/org/apache/aurora/gen/api.thrift     |  12 +-
 .../aurora/scheduler/ResourceAggregates.java    |   9 -
 .../aurora/scheduler/quota/QuotaInfo.java       |  67 +++++--
 .../aurora/scheduler/quota/QuotaManager.java    |  69 ++++---
 .../scheduler/thrift/ReadOnlySchedulerImpl.java |   7 +-
 .../apache/aurora/client/api/quota_check.py     |   2 +-
 .../python/apache/aurora/client/cli/quota.py    |   8 +-
 .../scheduler/assets/js/controllers.js          |  18 +-
 .../scheduler/quota/QuotaManagerImplTest.java   | 189 ++++++++++++-------
 .../storage/log/SnapshotStoreImplTest.java      |   6 +-
 .../aurora/scheduler/thrift/Fixtures.java       |  10 -
 .../thrift/ReadOnlySchedulerImplTest.java       |  30 +--
 .../python/apache/aurora/admin/test_admin.py    |   4 +-
 .../aurora/client/api/test_quota_check.py       |   2 +-
 .../apache/aurora/client/cli/test_quota.py      |  15 +-
 16 files changed, 270 insertions(+), 179 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/01e611ad/NEWS
----------------------------------------------------------------------
diff --git a/NEWS b/NEWS
index 83a1213..2edcea6 100644
--- a/NEWS
+++ b/NEWS
@@ -13,6 +13,7 @@
   container) to successfully launch Aurora executor. See
   https://github.com/apache/mesos/blob/ebcf8cc2f2f6c236f6e9315447c247e6348141e1/docs/getting-started.md
   for more details on Mesos runtime dependencies.
+- Resource quota is no longer consumed by production jobs with a dedicated constraint (AURORA-1457).
 
 0.9.0
 -----

http://git-wip-us.apache.org/repos/asf/aurora/blob/01e611ad/api/src/main/thrift/org/apache/aurora/gen/api.thrift
----------------------------------------------------------------------
diff --git a/api/src/main/thrift/org/apache/aurora/gen/api.thrift b/api/src/main/thrift/org/apache/aurora/gen/api.thrift
index 21137bb..24a2ca8 100644
--- a/api/src/main/thrift/org/apache/aurora/gen/api.thrift
+++ b/api/src/main/thrift/org/apache/aurora/gen/api.thrift
@@ -363,10 +363,14 @@ struct PopulateJobResult {
 struct GetQuotaResult {
   /** Total allocated resource quota. */
   1: ResourceAggregate quota
-  /** Resources consumed by production jobs. */
-  2: optional ResourceAggregate prodConsumption
-  /** Resources consumed by non-production jobs. */
-  3: optional ResourceAggregate nonProdConsumption
+  /** Resources consumed by production jobs from a shared resource pool. */
+  2: optional ResourceAggregate prodSharedConsumption
+  /** Resources consumed by non-production jobs from a shared resource pool. */
+  3: optional ResourceAggregate nonProdSharedConsumption
+  /** Resources consumed by production jobs from a dedicated resource pool. */
+  4: optional ResourceAggregate prodDedicatedConsumption
+  /** Resources consumed by non-production jobs from a dedicated resource pool. */
+  5: optional ResourceAggregate nonProdDedicatedConsumption
 }
 
 /** Wraps return results for the acquireLock API. */

http://git-wip-us.apache.org/repos/asf/aurora/blob/01e611ad/src/main/java/org/apache/aurora/scheduler/ResourceAggregates.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/ResourceAggregates.java b/src/main/java/org/apache/aurora/scheduler/ResourceAggregates.java
index df796fc..a6335c5 100644
--- a/src/main/java/org/apache/aurora/scheduler/ResourceAggregates.java
+++ b/src/main/java/org/apache/aurora/scheduler/ResourceAggregates.java
@@ -43,15 +43,6 @@ public final class ResourceAggregates {
   }
 
   /**
-   * Returns a quota with all resource vectors zeroed.
-   *
-   * @return A resource aggregate with all resource vectors zeroed.
-   */
-  public static IResourceAggregate none() {
-    return EMPTY;
-  }
-
-  /**
    * a * m.
    */
   public static IResourceAggregate scale(IResourceAggregate a, int m) {

http://git-wip-us.apache.org/repos/asf/aurora/blob/01e611ad/src/main/java/org/apache/aurora/scheduler/quota/QuotaInfo.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/quota/QuotaInfo.java b/src/main/java/org/apache/aurora/scheduler/quota/QuotaInfo.java
index b40ecd0..1df21b8 100644
--- a/src/main/java/org/apache/aurora/scheduler/quota/QuotaInfo.java
+++ b/src/main/java/org/apache/aurora/scheduler/quota/QuotaInfo.java
@@ -26,17 +26,23 @@ import static java.util.Objects.requireNonNull;
  */
 public class QuotaInfo {
   private final IResourceAggregate quota;
-  private final IResourceAggregate prodConsumption;
-  private final IResourceAggregate nonProdConsumption;
+  private final IResourceAggregate prodSharedConsumption;
+  private final IResourceAggregate prodDedicatedConsumption;
+  private final IResourceAggregate nonProdSharedConsumption;
+  private final IResourceAggregate nonProdDedicatedConsumption;
 
   QuotaInfo(
       IResourceAggregate quota,
-      IResourceAggregate prodConsumption,
-      IResourceAggregate nonProdConsumption) {
+      IResourceAggregate prodSharedConsumption,
+      IResourceAggregate prodDedicatedConsumption,
+      IResourceAggregate nonProdSharedConsumption,
+      IResourceAggregate nonProdDedicatedConsumption) {
 
     this.quota = requireNonNull(quota);
-    this.prodConsumption = requireNonNull(prodConsumption);
-    this.nonProdConsumption = requireNonNull(nonProdConsumption);
+    this.prodSharedConsumption = requireNonNull(prodSharedConsumption);
+    this.prodDedicatedConsumption = requireNonNull(prodDedicatedConsumption);
+    this.nonProdSharedConsumption = requireNonNull(nonProdSharedConsumption);
+    this.nonProdDedicatedConsumption = requireNonNull(nonProdDedicatedConsumption);
   }
 
   /**
@@ -49,21 +55,39 @@ public class QuotaInfo {
   }
 
   /**
-   * Quota consumed by production jobs.
+   * Quota consumed by production jobs from a shared resource pool.
    *
    * @return Production job consumption.
    */
-  public IResourceAggregate getProdConsumption() {
-    return prodConsumption;
+  public IResourceAggregate getProdSharedConsumption() {
+    return prodSharedConsumption;
   }
 
   /**
-   * Quota consumed by non-production jobs.
+   * Resources consumed by production jobs from a dedicated resource pool.
+   *
+   * @return Production dedicated job consumption.
+   */
+  public IResourceAggregate getProdDedicatedConsumption() {
+    return prodDedicatedConsumption;
+  }
+
+  /**
+   * Resources consumed by non-production jobs from a shared resource pool.
    *
    * @return Non production job consumption.
    */
-  public IResourceAggregate getNonProdConsumption() {
-    return nonProdConsumption;
+  public IResourceAggregate getNonProdSharedConsumption() {
+    return nonProdSharedConsumption;
+  }
+
+  /**
+   * Resources consumed by non-production jobs from a dedicated resource pool.
+   *
+   * @return Non production dedicated job consumption.
+   */
+  public IResourceAggregate getNonProdDedicatedConsumption() {
+    return nonProdDedicatedConsumption;
   }
 
   @Override
@@ -75,21 +99,30 @@ public class QuotaInfo {
     QuotaInfo other = (QuotaInfo) o;
 
     return Objects.equals(quota, other.quota)
-        && Objects.equals(prodConsumption, other.prodConsumption)
-        && Objects.equals(nonProdConsumption, other.nonProdConsumption);
+        && Objects.equals(prodSharedConsumption, other.prodSharedConsumption)
+        && Objects.equals(prodDedicatedConsumption, other.prodDedicatedConsumption)
+        && Objects.equals(nonProdSharedConsumption, other.nonProdSharedConsumption)
+        && Objects.equals(nonProdDedicatedConsumption, other.nonProdDedicatedConsumption);
   }
 
   @Override
   public int hashCode() {
-    return Objects.hash(quota, prodConsumption, nonProdConsumption);
+    return Objects.hash(
+        quota,
+        prodSharedConsumption,
+        prodDedicatedConsumption,
+        nonProdSharedConsumption,
+        nonProdDedicatedConsumption);
   }
 
   @Override
   public String toString() {
     return MoreObjects.toStringHelper(this)
         .add("quota", quota)
-        .add("prodConsumption", prodConsumption)
-        .add("nonProdConsumption", nonProdConsumption)
+        .add("prodSharedConsumption", prodSharedConsumption)
+        .add("prodDedicatedConsumption", prodDedicatedConsumption)
+        .add("nonProdSharedConsumption", nonProdSharedConsumption)
+        .add("nonProdDedicatedConsumption", nonProdDedicatedConsumption)
         .toString();
   }
 }

http://git-wip-us.apache.org/repos/asf/aurora/blob/01e611ad/src/main/java/org/apache/aurora/scheduler/quota/QuotaManager.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/quota/QuotaManager.java b/src/main/java/org/apache/aurora/scheduler/quota/QuotaManager.java
index 8787aea..49bf3c9 100644
--- a/src/main/java/org/apache/aurora/scheduler/quota/QuotaManager.java
+++ b/src/main/java/org/apache/aurora/scheduler/quota/QuotaManager.java
@@ -22,7 +22,6 @@ import com.google.common.base.Function;
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
-import com.google.common.base.Predicates;
 import com.google.common.collect.FluentIterable;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
@@ -36,6 +35,7 @@ import org.apache.aurora.scheduler.ResourceAggregates;
 import org.apache.aurora.scheduler.base.JobKeys;
 import org.apache.aurora.scheduler.base.Query;
 import org.apache.aurora.scheduler.base.Tasks;
+import org.apache.aurora.scheduler.configuration.ConfigurationManager;
 import org.apache.aurora.scheduler.storage.JobUpdateStore;
 import org.apache.aurora.scheduler.storage.Storage.MutableStoreProvider;
 import org.apache.aurora.scheduler.storage.Storage.StoreProvider;
@@ -55,6 +55,13 @@ import org.apache.aurora.scheduler.updater.Updates;
 
 import static java.util.Objects.requireNonNull;
 
+import static com.google.common.base.Predicates.and;
+import static com.google.common.base.Predicates.compose;
+import static com.google.common.base.Predicates.equalTo;
+import static com.google.common.base.Predicates.in;
+import static com.google.common.base.Predicates.not;
+import static com.google.common.base.Predicates.or;
+
 import static org.apache.aurora.scheduler.ResourceAggregates.EMPTY;
 import static org.apache.aurora.scheduler.quota.QuotaCheckResult.Result.SUFFICIENT_QUOTA;
 import static org.apache.aurora.scheduler.updater.Updates.getInstanceIds;
@@ -134,6 +141,15 @@ public interface QuotaManager {
    */
   class QuotaManagerImpl implements QuotaManager {
 
+    private static final Predicate<ITaskConfig> DEDICATED =
+        e -> ConfigurationManager.isDedicated(e.getConstraints());
+    private static final Predicate<ITaskConfig> PROD = ITaskConfig::isProduction;
+    private static final Predicate<ITaskConfig> PROD_SHARED = and(PROD, not(DEDICATED));
+    private static final Predicate<ITaskConfig> PROD_DEDICATED = and(PROD, DEDICATED);
+    private static final Predicate<ITaskConfig> NON_PROD_SHARED = and(not(PROD), not(DEDICATED));
+    private static final Predicate<ITaskConfig> NON_PROD_DEDICATED = and(not(PROD), DEDICATED);
+    private static final Predicate<ITaskConfig> NO_QUOTA_CHECK = or(PROD_DEDICATED, not(PROD));
+
     @Override
     public void saveQuota(
         final String ownerRole,
@@ -149,7 +165,7 @@ public interface QuotaManager {
       }
 
       QuotaInfo info = getQuotaInfo(ownerRole, Optional.absent(), storeProvider);
-      IResourceAggregate prodConsumption = info.getProdConsumption();
+      IResourceAggregate prodConsumption = info.getProdSharedConsumption();
       if (quota.getNumCpus() < prodConsumption.getNumCpus()
           || quota.getRamMb() < prodConsumption.getRamMb()
           || quota.getDiskMb() < prodConsumption.getDiskMb()) {
@@ -174,13 +190,13 @@ public interface QuotaManager {
         StoreProvider storeProvider) {
 
       Preconditions.checkArgument(instances >= 0);
-      if (!template.isProduction()) {
+      if (NO_QUOTA_CHECK.apply(template)) {
         return new QuotaCheckResult(SUFFICIENT_QUOTA);
       }
 
       QuotaInfo quotaInfo = getQuotaInfo(template.getJob().getRole(), storeProvider);
       IResourceAggregate requestedTotal =
-          add(quotaInfo.getProdConsumption(), scale(template, instances));
+          add(quotaInfo.getProdSharedConsumption(), scale(template, instances));
 
       return QuotaCheckResult.greaterOrEqual(quotaInfo.getQuota(), requestedTotal);
     }
@@ -189,7 +205,7 @@ public interface QuotaManager {
     public QuotaCheckResult checkJobUpdate(IJobUpdate jobUpdate, StoreProvider storeProvider) {
       requireNonNull(jobUpdate);
       if (!jobUpdate.getInstructions().isSetDesiredState()
-          || !jobUpdate.getInstructions().getDesiredState().getTask().isProduction()) {
+          || NO_QUOTA_CHECK.apply(jobUpdate.getInstructions().getDesiredState().getTask())) {
 
         return new QuotaCheckResult(SUFFICIENT_QUOTA);
       }
@@ -199,7 +215,9 @@ public interface QuotaManager {
           Optional.of(jobUpdate),
           storeProvider);
 
-      return QuotaCheckResult.greaterOrEqual(quotaInfo.getQuota(), quotaInfo.getProdConsumption());
+      return QuotaCheckResult.greaterOrEqual(
+          quotaInfo.getQuota(),
+          quotaInfo.getProdSharedConsumption());
     }
 
     @Override
@@ -222,7 +240,7 @@ public interface QuotaManager {
       // Calculate requested total as a sum of current prod consumption and a delta between
       // new and old cron templates.
       IResourceAggregate requestedTotal = add(
-          quotaInfo.getProdConsumption(),
+          quotaInfo.getProdSharedConsumption(),
           subtract(scale(cronConfig), oldResource));
 
       return QuotaCheckResult.greaterOrEqual(quotaInfo.getQuota(), requestedTotal);
@@ -262,45 +280,39 @@ public interface QuotaManager {
 
       Map<IJobKey, IJobConfiguration> cronTemplates =
           FluentIterable.from(storeProvider.getCronJobStore().fetchJobs())
-              .filter(Predicates.compose(Predicates.equalTo(role), JobKeys::getRole))
+              .filter(compose(equalTo(role), JobKeys::getRole))
               .uniqueIndex(IJobConfiguration::getKey);
 
-      IResourceAggregate prodConsumed = getConsumption(tasks, updates, cronTemplates, true);
-
-      IResourceAggregate nonProdConsumed = getConsumption(tasks, updates, cronTemplates, false);
-
-      IResourceAggregate quota =
-          storeProvider.getQuotaStore().fetchQuota(role).or(ResourceAggregates.none());
-
-      return new QuotaInfo(quota, prodConsumed, nonProdConsumed);
+      return new QuotaInfo(
+          storeProvider.getQuotaStore().fetchQuota(role).or(EMPTY),
+          getConsumption(tasks, updates, cronTemplates, PROD_SHARED),
+          getConsumption(tasks, updates, cronTemplates, PROD_DEDICATED),
+          getConsumption(tasks, updates, cronTemplates, NON_PROD_SHARED),
+          getConsumption(tasks, updates, cronTemplates, NON_PROD_DEDICATED));
     }
 
     private IResourceAggregate getConsumption(
         FluentIterable<IAssignedTask> tasks,
         Map<IJobKey, IJobUpdateInstructions> updatesByKey,
         Map<IJobKey, IJobConfiguration> cronTemplatesByKey,
-        boolean isProd) {
-
-      Predicate<ITaskConfig> prodFilter = isProd
-          ? ITaskConfig::isProduction
-          : Predicates.not(ITaskConfig::isProduction);
+        Predicate<ITaskConfig> filter) {
 
       FluentIterable<IAssignedTask> filteredTasks =
-          tasks.filter(Predicates.compose(prodFilter, IAssignedTask::getTask));
+          tasks.filter(compose(filter, IAssignedTask::getTask));
 
-      Predicate<IAssignedTask> excludeCron = Predicates.compose(
-          Predicates.not(Predicates.in(cronTemplatesByKey.keySet())),
+      Predicate<IAssignedTask> excludeCron = compose(
+          not(in(cronTemplatesByKey.keySet())),
           Tasks::getJob);
 
       IResourceAggregate nonCronConsumption = getNonCronConsumption(
           updatesByKey,
           filteredTasks.filter(excludeCron),
-          prodFilter);
+          filter);
 
       IResourceAggregate cronConsumption = getCronConsumption(
           Iterables.filter(
               cronTemplatesByKey.values(),
-              Predicates.compose(prodFilter, IJobConfiguration::getTaskConfig)),
+              compose(filter, IJobConfiguration::getTaskConfig)),
           filteredTasks.transform(IAssignedTask::getTask));
 
       return add(nonCronConsumption, cronConsumption);
@@ -327,7 +339,7 @@ public interface QuotaManager {
           .transform(IAssignedTask::getTask));
 
       final Predicate<IInstanceTaskConfig> instanceFilter =
-          Predicates.compose(configFilter, IInstanceTaskConfig::getTask);
+          compose(configFilter, IInstanceTaskConfig::getTask);
 
       IResourceAggregate updateConsumption =
           addAll(Iterables.transform(updatesByKey.values(), updateResources(instanceFilter)));
@@ -445,7 +457,8 @@ public interface QuotaManager {
      * <p/>
      * NOTE: In case of a job update converting the job production bit (i.e. prod -> non-prod or
      *       non-prod -> prod), only the matching state is counted towards consumption. For example,
-     *       prod -> non-prod AND {@code prodConsumption=True}: only the initial state is accounted.
+     *       prod -> non-prod AND {@code prodSharedConsumption=True}: only the initial state
+     *       is accounted.
      */
     private static Function<IJobUpdateInstructions, IResourceAggregate> updateResources(
         final Predicate<IInstanceTaskConfig> instanceFilter) {

http://git-wip-us.apache.org/repos/asf/aurora/blob/01e611ad/src/main/java/org/apache/aurora/scheduler/thrift/ReadOnlySchedulerImpl.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/thrift/ReadOnlySchedulerImpl.java b/src/main/java/org/apache/aurora/scheduler/thrift/ReadOnlySchedulerImpl.java
index 3d89e43..13b5c22 100644
--- a/src/main/java/org/apache/aurora/scheduler/thrift/ReadOnlySchedulerImpl.java
+++ b/src/main/java/org/apache/aurora/scheduler/thrift/ReadOnlySchedulerImpl.java
@@ -300,8 +300,11 @@ class ReadOnlySchedulerImpl implements ReadOnlyScheduler.Iface {
       public Response apply(StoreProvider storeProvider) {
         QuotaInfo quotaInfo = quotaManager.getQuotaInfo(ownerRole, storeProvider);
         GetQuotaResult result = new GetQuotaResult(quotaInfo.getQuota().newBuilder())
-            .setProdConsumption(quotaInfo.getProdConsumption().newBuilder())
-            .setNonProdConsumption(quotaInfo.getNonProdConsumption().newBuilder());
+            .setProdSharedConsumption(quotaInfo.getProdSharedConsumption().newBuilder())
+            .setProdDedicatedConsumption(quotaInfo.getProdDedicatedConsumption().newBuilder())
+            .setNonProdSharedConsumption(quotaInfo.getNonProdSharedConsumption().newBuilder())
+            .setNonProdDedicatedConsumption(
+                quotaInfo.getNonProdDedicatedConsumption().newBuilder());
 
         return ok(Result.getQuotaResult(result));
       }

http://git-wip-us.apache.org/repos/asf/aurora/blob/01e611ad/src/main/python/apache/aurora/client/api/quota_check.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/api/quota_check.py b/src/main/python/apache/aurora/client/api/quota_check.py
index 75406ac..a3252e5 100644
--- a/src/main/python/apache/aurora/client/api/quota_check.py
+++ b/src/main/python/apache/aurora/client/api/quota_check.py
@@ -101,7 +101,7 @@ class QuotaCheck(object):
       return resp
 
     allocated = CapacityRequest(resp.result.getQuotaResult.quota)
-    consumed = CapacityRequest(resp.result.getQuotaResult.prodConsumption)
+    consumed = CapacityRequest(resp.result.getQuotaResult.prodSharedConsumption)
     requested = acquired - released
     effective = allocated - consumed - requested
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/01e611ad/src/main/python/apache/aurora/client/cli/quota.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/quota.py b/src/main/python/apache/aurora/client/cli/quota.py
index e8aa010..fe29333 100644
--- a/src/main/python/apache/aurora/client/cli/quota.py
+++ b/src/main/python/apache/aurora/client/cli/quota.py
@@ -56,12 +56,12 @@ class GetQuotaCmd(Verb):
       quota_result = quota_resp.result.getQuotaResult
       result = ['Allocated:']
       result += get_quota_str(quota_result.quota)
-      if quota_result.prodConsumption:
+      if quota_result.prodSharedConsumption:
         result.append('Production resources consumed:')
-        result += get_quota_str(quota_result.prodConsumption)
-      if quota_result.nonProdConsumption:
+        result += get_quota_str(quota_result.prodSharedConsumption)
+      if quota_result.nonProdSharedConsumption:
         result.append('Non-production resources consumed:')
-        result += get_quota_str(quota_result.nonProdConsumption)
+        result += get_quota_str(quota_result.nonProdSharedConsumption)
       return '\n'.join(result)
 
   def execute(self, context):

http://git-wip-us.apache.org/repos/asf/aurora/blob/01e611ad/src/main/resources/scheduler/assets/js/controllers.js
----------------------------------------------------------------------
diff --git a/src/main/resources/scheduler/assets/js/controllers.js b/src/main/resources/scheduler/assets/js/controllers.js
index 4acab10..9436e9b 100644
--- a/src/main/resources/scheduler/assets/js/controllers.js
+++ b/src/main/resources/scheduler/assets/js/controllers.js
@@ -160,9 +160,9 @@
 
       $scope.resourcesTableColumns = [
         {label: 'Resource', map: 'resource'},
-        {label: 'Production Consumption', map: 'prodConsumption'},
+        {label: 'Production Consumption', map: 'prodSharedConsumption'},
         {label: 'Quota', map: 'quota'},
-        {label: 'Non-Production Consumption', map: 'nonProdConsumption'}
+        {label: 'Non-Production Consumption', map: 'nonProdSharedConsumption'}
       ];
 
       $scope.resourcesTableConfig = summaryTableConfig;
@@ -183,21 +183,23 @@
         return [
           {
             resource: 'CPU',
-            prodConsumption: $filter('toCores')(consumption.prodConsumption.numCpus),
+            prodSharedConsumption: $filter('toCores')(consumption.prodSharedConsumption.numCpus),
             quota: $filter('toCores')(consumption.quota.numCpus),
-            nonProdConsumption: $filter('toCores')(consumption.nonProdConsumption.numCpus)
+            nonProdSharedConsumption:
+              $filter('toCores')(consumption.nonProdSharedConsumption.numCpus)
           },
           {
             resource: 'RAM',
-            prodConsumption: $filter('scaleMb')(consumption.prodConsumption.ramMb),
+            prodSharedConsumption: $filter('scaleMb')(consumption.prodSharedConsumption.ramMb),
             quota: $filter('scaleMb')(consumption.quota.ramMb),
-            nonProdConsumption: $filter('scaleMb')(consumption.nonProdConsumption.ramMb)
+            nonProdSharedConsumption: $filter('scaleMb')(consumption.nonProdSharedConsumption.ramMb)
           },
           {
             resource: 'Disk',
-            prodConsumption: $filter('scaleMb')(consumption.prodConsumption.diskMb),
+            prodSharedConsumption: $filter('scaleMb')(consumption.prodSharedConsumption.diskMb),
             quota: $filter('scaleMb')(consumption.quota.diskMb),
-            nonProdConsumption: $filter('scaleMb')(consumption.nonProdConsumption.diskMb)
+            nonProdSharedConsumption:
+              $filter('scaleMb')(consumption.nonProdSharedConsumption.diskMb)
           }
         ];
       }

http://git-wip-us.apache.org/repos/asf/aurora/blob/01e611ad/src/test/java/org/apache/aurora/scheduler/quota/QuotaManagerImplTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/quota/QuotaManagerImplTest.java b/src/test/java/org/apache/aurora/scheduler/quota/QuotaManagerImplTest.java
index db60cd2..83a8abe 100644
--- a/src/test/java/org/apache/aurora/scheduler/quota/QuotaManagerImplTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/quota/QuotaManagerImplTest.java
@@ -21,6 +21,7 @@ import com.google.common.collect.ImmutableSet;
 
 import org.apache.aurora.common.testing.easymock.EasyMockTest;
 import org.apache.aurora.gen.AssignedTask;
+import org.apache.aurora.gen.Constraint;
 import org.apache.aurora.gen.Identity;
 import org.apache.aurora.gen.InstanceTaskConfig;
 import org.apache.aurora.gen.JobConfiguration;
@@ -34,7 +35,8 @@ import org.apache.aurora.gen.ResourceAggregate;
 import org.apache.aurora.gen.ScheduleStatus;
 import org.apache.aurora.gen.ScheduledTask;
 import org.apache.aurora.gen.TaskConfig;
-import org.apache.aurora.scheduler.ResourceAggregates;
+import org.apache.aurora.gen.TaskConstraint;
+import org.apache.aurora.gen.ValueConstraint;
 import org.apache.aurora.scheduler.base.JobKeys;
 import org.apache.aurora.scheduler.base.Query;
 import org.apache.aurora.scheduler.quota.QuotaManager.QuotaException;
@@ -54,6 +56,7 @@ import org.easymock.IExpectationSetters;
 import org.junit.Before;
 import org.junit.Test;
 
+import static org.apache.aurora.scheduler.ResourceAggregates.EMPTY;
 import static org.apache.aurora.scheduler.quota.QuotaCheckResult.Result.INSUFFICIENT_QUOTA;
 import static org.apache.aurora.scheduler.quota.QuotaCheckResult.Result.SUFFICIENT_QUOTA;
 import static org.apache.aurora.scheduler.quota.QuotaManager.QuotaManagerImpl.updateQuery;
@@ -90,28 +93,30 @@ public class QuotaManagerImplTest extends EasyMockTest {
 
   @Test
   public void testGetQuotaInfo() {
-    IScheduledTask prodTask = createProdTask("foo", 3, 3, 3);
-    IScheduledTask nonProdTask = createTask("bar", "id1", 2, 2, 2, false, 0);
+    IScheduledTask prodSharedTask = prodTask("foo1", 3, 3, 3);
+    IScheduledTask prodDedicatedTask = prodDedicatedTask("foo2", 5, 5, 5);
+    IScheduledTask nonProdSharedTask = nonProdTask("bar1", 2, 2, 2);
+    IScheduledTask nonProdDedicatedTask = nonProdDedicatedTask("bar2", 7, 7, 7);
     IResourceAggregate quota = IResourceAggregate.build(new ResourceAggregate(4, 4, 4));
 
     expectQuota(quota);
-    expectTasks(prodTask, nonProdTask);
+    expectTasks(prodSharedTask, nonProdSharedTask, prodDedicatedTask, nonProdDedicatedTask);
     expectJobUpdates(taskConfig(1, 1, 1, true), taskConfig(1, 1, 1, true));
     expectCronJobs(
-        createJob(createProdTask("pc", 1, 1, 1), 2),
-        createJob(createNonProdTask("npc", 7, 7, 7), 1));
+        createJob(prodTask("pc", 1, 1, 1), 2),
+        createJob(nonProdTask("npc", 7, 7, 7), 1));
 
     control.replay();
 
     assertEquals(
-        new QuotaInfo(from(4, 4, 4), from(6, 6, 6), from(9, 9, 9)),
+        new QuotaInfo(from(4, 4, 4), from(6, 6, 6), from(5, 5, 5), from(9, 9, 9), from(7, 7, 7)),
         quotaManager.getQuotaInfo(ROLE, storeProvider));
   }
 
   @Test
   public void testGetQuotaInfoWithCronTasks() {
-    IScheduledTask prodTask = createProdTask("pc", 6, 6, 6);
-    IScheduledTask nonProdTask = createProdTask("npc", 7, 7, 7);
+    IScheduledTask prodTask = prodTask("pc", 6, 6, 6);
+    IScheduledTask nonProdTask = prodTask("npc", 7, 7, 7);
     IResourceAggregate quota = IResourceAggregate.build(new ResourceAggregate(4, 4, 4));
 
     expectQuota(quota);
@@ -119,33 +124,33 @@ public class QuotaManagerImplTest extends EasyMockTest {
     expectJobUpdates(taskConfig(1, 1, 1, true), taskConfig(1, 1, 1, true));
 
     final String pcRole = "pc-role";
-    ScheduledTask ignoredProdTask = createProdTask(pcRole, 20, 20, 20).newBuilder();
+    ScheduledTask ignoredProdTask = prodTask(pcRole, 20, 20, 20).newBuilder();
     ignoredProdTask.getAssignedTask().getTask()
         .setOwner(new Identity(pcRole, "ignored"))
         .setJob(new JobKey(pcRole, ENV, pcRole));
 
     final String npcRole = "npc-role";
-    ScheduledTask ignoredNonProdTask = createNonProdTask(npcRole, 20, 20, 20).newBuilder();
+    ScheduledTask ignoredNonProdTask = nonProdTask(npcRole, 20, 20, 20).newBuilder();
     ignoredNonProdTask.getAssignedTask().getTask()
         .setOwner(new Identity(npcRole, "ignored"))
         .setJob(new JobKey(npcRole, ENV, npcRole));
 
     expectCronJobs(
-        createJob(createProdTask("pc", 3, 3, 3), 1),
-        createJob(createNonProdTask("npc", 5, 5, 5), 2),
+        createJob(prodTask("pc", 3, 3, 3), 1),
+        createJob(nonProdTask("npc", 5, 5, 5), 2),
         createJob(IScheduledTask.build(ignoredProdTask), 2),
         createJob(IScheduledTask.build(ignoredNonProdTask), 3));
 
     control.replay();
 
     assertEquals(
-        new QuotaInfo(from(4, 4, 4), from(7, 7, 7), from(10, 10, 10)),
+        new QuotaInfo(from(4, 4, 4), from(7, 7, 7), EMPTY, from(10, 10, 10), EMPTY),
         quotaManager.getQuotaInfo(ROLE, storeProvider));
   }
 
   @Test
   public void testGetQuotaInfoPartialUpdate() {
-    IScheduledTask prodTask = createProdTask("foo", 3, 3, 3);
+    IScheduledTask prodTask = prodTask("foo", 3, 3, 3);
     IScheduledTask updatingProdTask = createTask(JOB_NAME, "id1", 3, 3, 3, true, 1);
     IScheduledTask updatingFilteredProdTask = createTask(JOB_NAME, "id0", 3, 3, 3, true, 0);
     IScheduledTask nonProdTask = createTask("bar", "id1", 2, 2, 2, false, 0);
@@ -160,7 +165,7 @@ public class QuotaManagerImplTest extends EasyMockTest {
 
     // Expected consumption from: prodTask + updatingProdTask + job update.
     assertEquals(
-        new QuotaInfo(from(4, 4, 4), from(7, 7, 7), from(2, 2, 2)),
+        new QuotaInfo(from(4, 4, 4), from(7, 7, 7), EMPTY, from(2, 2, 2), EMPTY),
         quotaManager.getQuotaInfo(ROLE, storeProvider));
   }
 
@@ -176,14 +181,14 @@ public class QuotaManagerImplTest extends EasyMockTest {
     control.replay();
 
     assertEquals(
-        new QuotaInfo(from(4, 4, 4), from(0, 0, 0), from(0, 0, 0)),
+        new QuotaInfo(from(4, 4, 4), from(0, 0, 0), EMPTY, from(0, 0, 0), EMPTY),
         quotaManager.getQuotaInfo(ROLE, storeProvider));
   }
 
   @Test
   public void testCheckQuotaPasses() {
     expectQuota(IResourceAggregate.build(new ResourceAggregate(4, 4, 4)));
-    expectTasks(createProdTask("foo", 2, 2, 2));
+    expectTasks(prodTask("foo", 2, 2, 2));
     expectJobUpdates(taskConfig(1, 1, 1, true), taskConfig(1, 1, 1, true));
     expectNoCronJobs();
 
@@ -211,7 +216,7 @@ public class QuotaManagerImplTest extends EasyMockTest {
   @Test
   public void testCheckQuotaPassesNoUpdates() {
     expectQuota(IResourceAggregate.build(new ResourceAggregate(4, 4, 4)));
-    expectTasks(createProdTask("foo", 2, 2, 2));
+    expectTasks(prodTask("foo", 2, 2, 2));
     expectNoJobUpdates();
     expectNoCronJobs();
 
@@ -239,7 +244,7 @@ public class QuotaManagerImplTest extends EasyMockTest {
   @Test
   public void testCheckQuotaPassesNonProdUnaccounted() {
     expectQuota(IResourceAggregate.build(new ResourceAggregate(4, 4, 4)));
-    expectTasks(createProdTask("foo", 2, 2, 2), createTask("bar", "id2", 5, 5, 5, false, 0));
+    expectTasks(prodTask("foo", 2, 2, 2), createTask("bar", "id2", 5, 5, 5, false, 0));
 
     expectNoJobUpdates();
     expectNoCronJobs();
@@ -252,6 +257,17 @@ public class QuotaManagerImplTest extends EasyMockTest {
   }
 
   @Test
+  public void testCheckQuotaPassesDedicatedUnaccounted() {
+    control.replay();
+
+    QuotaCheckResult checkQuota = quotaManager.checkInstanceAddition(
+        prodDedicatedTask("dedicatedJob", 1, 1, 1).getAssignedTask().getTask(),
+        1,
+        storeProvider);
+    assertEquals(SUFFICIENT_QUOTA, checkQuota.getResult());
+  }
+
+  @Test
   public void testCheckQuotaSkippedForNonProdRequest() {
     control.replay();
 
@@ -279,7 +295,7 @@ public class QuotaManagerImplTest extends EasyMockTest {
   @Test
   public void testCheckQuotaExceedsCpu() {
     expectQuota(IResourceAggregate.build(new ResourceAggregate(4, 4, 4)));
-    expectTasks(createProdTask("foo", 3, 3, 3));
+    expectTasks(prodTask("foo", 3, 3, 3));
     expectNoJobUpdates();
     expectNoCronJobs();
 
@@ -294,7 +310,7 @@ public class QuotaManagerImplTest extends EasyMockTest {
   @Test
   public void testCheckQuotaExceedsRam() {
     expectQuota(IResourceAggregate.build(new ResourceAggregate(4, 4, 4)));
-    expectTasks(createProdTask("foo", 3, 3, 3));
+    expectTasks(prodTask("foo", 3, 3, 3));
     expectNoJobUpdates();
     expectNoCronJobs();
 
@@ -309,7 +325,7 @@ public class QuotaManagerImplTest extends EasyMockTest {
   @Test
   public void testCheckQuotaExceedsDisk() {
     expectQuota(IResourceAggregate.build(new ResourceAggregate(4, 4, 4)));
-    expectTasks(createProdTask("foo", 3, 3, 3));
+    expectTasks(prodTask("foo", 3, 3, 3));
     expectNoJobUpdates();
     expectNoCronJobs();
 
@@ -327,8 +343,8 @@ public class QuotaManagerImplTest extends EasyMockTest {
     expectNoTasks().times(2);
     expectNoJobUpdates().times(2);
     expectCronJobs(
-        createJob(createProdTask("pc", 4, 4, 4), 1),
-        createJob(createNonProdTask("npc", 7, 7, 7), 1)).times(2);
+        createJob(prodTask("pc", 4, 4, 4), 1),
+        createJob(nonProdTask("npc", 7, 7, 7), 1)).times(2);
 
     control.replay();
 
@@ -336,14 +352,14 @@ public class QuotaManagerImplTest extends EasyMockTest {
         quotaManager.checkInstanceAddition(taskConfig(2, 2, 2, true), 1, storeProvider);
     assertEquals(INSUFFICIENT_QUOTA, checkQuota.getResult());
     assertEquals(
-        new QuotaInfo(from(5, 5, 5), from(4, 4, 4), from(7, 7, 7)),
+        new QuotaInfo(from(5, 5, 5), from(4, 4, 4), EMPTY, from(7, 7, 7), EMPTY),
         quotaManager.getQuotaInfo(ROLE, storeProvider));
   }
 
   @Test
   public void testCheckQuotaUpdatingTasksFilteredOut() {
     expectQuota(IResourceAggregate.build(new ResourceAggregate(5, 5, 5))).times(2);
-    expectTasks(createProdTask("foo", 2, 2, 2), createTask(JOB_NAME, "id2", 3, 3, 3, true, 0))
+    expectTasks(prodTask("foo", 2, 2, 2), createTask(JOB_NAME, "id2", 3, 3, 3, true, 0))
         .times(2);
 
     expectJobUpdates(taskConfig(1, 1, 1, true), taskConfig(2, 2, 2, true), 2);
@@ -355,14 +371,14 @@ public class QuotaManagerImplTest extends EasyMockTest {
         quotaManager.checkInstanceAddition(taskConfig(1, 1, 1, true), 1, storeProvider);
     assertEquals(SUFFICIENT_QUOTA, checkQuota.getResult());
     assertEquals(
-        new QuotaInfo(from(5, 5, 5), from(4, 4, 4), from(0, 0, 0)),
+        new QuotaInfo(from(5, 5, 5), from(4, 4, 4), EMPTY, from(0, 0, 0), EMPTY),
         quotaManager.getQuotaInfo(ROLE, storeProvider));
   }
 
   @Test
   public void testCheckQuotaNonProdUpdatesUnaccounted() {
     expectQuota(IResourceAggregate.build(new ResourceAggregate(5, 5, 5))).times(2);
-    expectTasks(createProdTask("foo", 2, 2, 2), createProdTask("bar", 2, 2, 2)).times(2);
+    expectTasks(prodTask("foo", 2, 2, 2), prodTask("bar", 2, 2, 2)).times(2);
 
     expectJobUpdates(taskConfig(8, 8, 8, false), taskConfig(4, 4, 4, false), 2);
     expectNoCronJobs().times(2);
@@ -373,14 +389,14 @@ public class QuotaManagerImplTest extends EasyMockTest {
         quotaManager.checkInstanceAddition(taskConfig(1, 1, 1, true), 1, storeProvider);
     assertEquals(SUFFICIENT_QUOTA, checkQuota.getResult());
     assertEquals(
-        new QuotaInfo(from(5, 5, 5), from(4, 4, 4), from(8, 8, 8)),
+        new QuotaInfo(from(5, 5, 5), from(4, 4, 4), EMPTY, from(8, 8, 8), EMPTY),
         quotaManager.getQuotaInfo(ROLE, storeProvider));
   }
 
   @Test
   public void testCheckQuotaProdToNonUpdateUnaccounted() {
     expectQuota(IResourceAggregate.build(new ResourceAggregate(5, 5, 5))).times(2);
-    expectTasks(createProdTask("foo", 2, 2, 2), createProdTask("bar", 1, 1, 1)).times(2);
+    expectTasks(prodTask("foo", 2, 2, 2), prodTask("bar", 1, 1, 1)).times(2);
 
     expectJobUpdates(taskConfig(1, 1, 1, true), taskConfig(7, 7, 7, false), 2);
     expectNoCronJobs().times(2);
@@ -391,14 +407,14 @@ public class QuotaManagerImplTest extends EasyMockTest {
         quotaManager.checkInstanceAddition(taskConfig(1, 1, 1, true), 1, storeProvider);
     assertEquals(SUFFICIENT_QUOTA, checkQuota.getResult());
     assertEquals(
-        new QuotaInfo(from(5, 5, 5), from(4, 4, 4), from(7, 7, 7)),
+        new QuotaInfo(from(5, 5, 5), from(4, 4, 4), EMPTY, from(7, 7, 7), EMPTY),
         quotaManager.getQuotaInfo(ROLE, storeProvider));
   }
 
   @Test
   public void testCheckQuotaNonToProdUpdateExceedsQuota() {
     expectQuota(IResourceAggregate.build(new ResourceAggregate(5, 5, 5))).times(2);
-    expectTasks(createProdTask("foo", 2, 2, 2), createProdTask("bar", 2, 2, 2)).times(2);
+    expectTasks(prodTask("foo", 2, 2, 2), prodTask("bar", 2, 2, 2)).times(2);
 
     expectJobUpdates(taskConfig(1, 1, 1, false), taskConfig(1, 1, 1, true), 2);
     expectNoCronJobs().times(2);
@@ -409,14 +425,14 @@ public class QuotaManagerImplTest extends EasyMockTest {
         quotaManager.checkInstanceAddition(taskConfig(1, 1, 1, true), 1, storeProvider);
     assertEquals(INSUFFICIENT_QUOTA, checkQuota.getResult());
     assertEquals(
-        new QuotaInfo(from(5, 5, 5), from(5, 5, 5), from(1, 1, 1)),
+        new QuotaInfo(from(5, 5, 5), from(5, 5, 5), EMPTY, from(1, 1, 1), EMPTY),
         quotaManager.getQuotaInfo(ROLE, storeProvider));
   }
 
   @Test
   public void testCheckQuotaOldJobUpdateConfigMatters() {
     expectQuota(IResourceAggregate.build(new ResourceAggregate(6, 6, 6))).times(2);
-    expectTasks(createProdTask("foo", 2, 2, 2), createProdTask("bar", 2, 2, 2)).times(2);
+    expectTasks(prodTask("foo", 2, 2, 2), prodTask("bar", 2, 2, 2)).times(2);
     expectJobUpdates(taskConfig(2, 2, 2, true), taskConfig(1, 1, 1, true), 2);
     expectNoCronJobs().times(2);
 
@@ -426,14 +442,14 @@ public class QuotaManagerImplTest extends EasyMockTest {
         quotaManager.checkInstanceAddition(taskConfig(1, 1, 1, true), 1, storeProvider);
     assertEquals(INSUFFICIENT_QUOTA, checkQuota.getResult());
     assertEquals(
-        new QuotaInfo(from(6, 6, 6), from(6, 6, 6), from(0, 0, 0)),
+        new QuotaInfo(from(6, 6, 6), from(6, 6, 6), EMPTY, from(0, 0, 0), EMPTY),
         quotaManager.getQuotaInfo(ROLE, storeProvider));
   }
 
   @Test
   public void testCheckQuotaUpdateAddsInstances() {
     expectQuota(IResourceAggregate.build(new ResourceAggregate(6, 6, 6))).times(2);
-    expectTasks(createProdTask("foo", 2, 2, 2), createProdTask("bar", 2, 2, 2)).times(2);
+    expectTasks(prodTask("foo", 2, 2, 2), prodTask("bar", 2, 2, 2)).times(2);
     expectJobUpdates(taskConfig(1, 1, 1, true), 1, taskConfig(1, 1, 1, true), 2, 2);
     expectNoCronJobs().times(2);
 
@@ -443,14 +459,14 @@ public class QuotaManagerImplTest extends EasyMockTest {
         quotaManager.checkInstanceAddition(taskConfig(1, 1, 1, true), 1, storeProvider);
     assertEquals(INSUFFICIENT_QUOTA, checkQuota.getResult());
     assertEquals(
-        new QuotaInfo(from(6, 6, 6), from(6, 6, 6), from(0, 0, 0)),
+        new QuotaInfo(from(6, 6, 6), from(6, 6, 6), EMPTY, from(0, 0, 0), EMPTY),
         quotaManager.getQuotaInfo(ROLE, storeProvider));
   }
 
   @Test
   public void testCheckQuotaUpdateRemovesInstances() {
     expectQuota(IResourceAggregate.build(new ResourceAggregate(6, 6, 6))).times(2);
-    expectTasks(createProdTask("foo", 2, 2, 2), createProdTask("bar", 2, 2, 2)).times(2);
+    expectTasks(prodTask("foo", 2, 2, 2), prodTask("bar", 2, 2, 2)).times(2);
     expectJobUpdates(taskConfig(1, 1, 1, true), 2, taskConfig(1, 1, 1, true), 1, 2);
     expectNoCronJobs().times(2);
 
@@ -460,14 +476,14 @@ public class QuotaManagerImplTest extends EasyMockTest {
         quotaManager.checkInstanceAddition(taskConfig(1, 1, 1, true), 1, storeProvider);
     assertEquals(INSUFFICIENT_QUOTA, checkQuota.getResult());
     assertEquals(
-        new QuotaInfo(from(6, 6, 6), from(6, 6, 6), from(0, 0, 0)),
+        new QuotaInfo(from(6, 6, 6), from(6, 6, 6), EMPTY, from(0, 0, 0), EMPTY),
         quotaManager.getQuotaInfo(ROLE, storeProvider));
   }
 
   @Test
   public void testCheckQuotaUpdateInitialConfigsUsedForFiltering() {
     expectQuota(IResourceAggregate.build(new ResourceAggregate(6, 6, 6))).times(2);
-    expectTasks(createProdTask("foo", 2, 2, 2), createProdTask(JOB_NAME, 2, 2, 2)).times(2);
+    expectTasks(prodTask("foo", 2, 2, 2), prodTask(JOB_NAME, 2, 2, 2)).times(2);
 
     ITaskConfig config = taskConfig(2, 2, 2, true);
     List<IJobUpdateSummary> summaries = buildJobUpdateSummaries(UPDATE_KEY);
@@ -489,14 +505,14 @@ public class QuotaManagerImplTest extends EasyMockTest {
         quotaManager.checkInstanceAddition(taskConfig(1, 1, 1, true), 1, storeProvider);
     assertEquals(SUFFICIENT_QUOTA, checkQuota.getResult());
     assertEquals(
-        new QuotaInfo(from(6, 6, 6), from(4, 4, 4), from(0, 0, 0)),
+        new QuotaInfo(from(6, 6, 6), from(4, 4, 4), EMPTY, from(0, 0, 0), EMPTY),
         quotaManager.getQuotaInfo(ROLE, storeProvider));
   }
 
   @Test
   public void testCheckQuotaUpdateDesiredConfigsUsedForFiltering() {
     expectQuota(IResourceAggregate.build(new ResourceAggregate(6, 6, 6))).times(2);
-    expectTasks(createProdTask("foo", 2, 2, 2), createProdTask(JOB_NAME, 2, 2, 2)).times(2);
+    expectTasks(prodTask("foo", 2, 2, 2), prodTask(JOB_NAME, 2, 2, 2)).times(2);
 
     ITaskConfig config = taskConfig(2, 2, 2, true);
     List<IJobUpdateSummary> summaries = buildJobUpdateSummaries(UPDATE_KEY);
@@ -518,14 +534,14 @@ public class QuotaManagerImplTest extends EasyMockTest {
         quotaManager.checkInstanceAddition(taskConfig(1, 1, 1, true), 1, storeProvider);
     assertEquals(SUFFICIENT_QUOTA, checkQuota.getResult());
     assertEquals(
-        new QuotaInfo(from(6, 6, 6), from(4, 4, 4), from(0, 0, 0)),
+        new QuotaInfo(from(6, 6, 6), from(4, 4, 4), EMPTY, from(0, 0, 0), EMPTY),
         quotaManager.getQuotaInfo(ROLE, storeProvider));
   }
 
   @Test
   public void testCheckQuotaNoDesiredState() {
     expectQuota(IResourceAggregate.build(new ResourceAggregate(6, 6, 6))).times(2);
-    expectTasks(createProdTask("foo", 2, 2, 2), createProdTask("bar", 2, 2, 2)).times(2);
+    expectTasks(prodTask("foo", 2, 2, 2), prodTask("bar", 2, 2, 2)).times(2);
 
     ITaskConfig config = taskConfig(2, 2, 2, true);
     List<IJobUpdateSummary> summaries = buildJobUpdateSummaries(UPDATE_KEY);
@@ -547,7 +563,7 @@ public class QuotaManagerImplTest extends EasyMockTest {
         quotaManager.checkInstanceAddition(taskConfig(1, 1, 1, true), 1, storeProvider);
     assertEquals(INSUFFICIENT_QUOTA, checkQuota.getResult());
     assertEquals(
-        new QuotaInfo(from(6, 6, 6), from(6, 6, 6), from(0, 0, 0)),
+        new QuotaInfo(from(6, 6, 6), from(6, 6, 6), EMPTY, from(0, 0, 0), EMPTY),
         quotaManager.getQuotaInfo(ROLE, storeProvider));
   }
 
@@ -555,7 +571,7 @@ public class QuotaManagerImplTest extends EasyMockTest {
   public void testCheckQuotaNewInPlaceUpdate() {
     expectQuota(IResourceAggregate.build(new ResourceAggregate(6, 6, 6))).times(2);
     expectTasks(
-        createProdTask("foo", 2, 2, 2),
+        prodTask("foo", 2, 2, 2),
         createTask(JOB_NAME, "id1", 2, 2, 2, true, 0),
         createTask(JOB_NAME, "id12", 2, 2, 2, true, 12)).times(2);
     expectNoJobUpdates().times(2);
@@ -575,14 +591,14 @@ public class QuotaManagerImplTest extends EasyMockTest {
     QuotaCheckResult checkQuota = quotaManager.checkJobUpdate(update, storeProvider);
     assertEquals(SUFFICIENT_QUOTA, checkQuota.getResult());
     assertEquals(
-        new QuotaInfo(from(6, 6, 6), from(6, 6, 6), from(0, 0, 0)),
+        new QuotaInfo(from(6, 6, 6), from(6, 6, 6), EMPTY, from(0, 0, 0), EMPTY),
         quotaManager.getQuotaInfo(ROLE, storeProvider));
   }
 
   @Test
   public void testCheckQuotaNewUpdateAddsInstances() {
     expectQuota(IResourceAggregate.build(new ResourceAggregate(6, 6, 6))).times(2);
-    expectTasks(createProdTask("foo", 2, 2, 2), createProdTask(JOB_NAME, 2, 2, 2)).times(2);
+    expectTasks(prodTask("foo", 2, 2, 2), prodTask(JOB_NAME, 2, 2, 2)).times(2);
     expectNoJobUpdates().times(2);
 
     ITaskConfig config = taskConfig(2, 2, 2, true);
@@ -600,7 +616,7 @@ public class QuotaManagerImplTest extends EasyMockTest {
     QuotaCheckResult checkQuota = quotaManager.checkJobUpdate(update, storeProvider);
     assertEquals(INSUFFICIENT_QUOTA, checkQuota.getResult());
     assertEquals(
-        new QuotaInfo(from(6, 6, 6), from(4, 4, 4), from(0, 0, 0)),
+        new QuotaInfo(from(6, 6, 6), from(4, 4, 4), EMPTY, from(0, 0, 0), EMPTY),
         quotaManager.getQuotaInfo(ROLE, storeProvider));
   }
 
@@ -608,7 +624,7 @@ public class QuotaManagerImplTest extends EasyMockTest {
   public void testCheckQuotaNewUpdateRemovesInstances() {
     expectQuota(IResourceAggregate.build(new ResourceAggregate(6, 6, 6))).times(2);
     expectTasks(
-        createProdTask("foo", 2, 2, 2),
+        prodTask("foo", 2, 2, 2),
         createTask(JOB_NAME, "id1", 2, 2, 2, true, 0),
         createTask(JOB_NAME, "id2", 2, 2, 2, true, 1)).times(2);
     expectNoJobUpdates().times(2);
@@ -628,7 +644,7 @@ public class QuotaManagerImplTest extends EasyMockTest {
     QuotaCheckResult checkQuota = quotaManager.checkJobUpdate(update, storeProvider);
     assertEquals(SUFFICIENT_QUOTA, checkQuota.getResult());
     assertEquals(
-        new QuotaInfo(from(6, 6, 6), from(6, 6, 6), from(0, 0, 0)),
+        new QuotaInfo(from(6, 6, 6), from(6, 6, 6), EMPTY, from(0, 0, 0), EMPTY),
         quotaManager.getQuotaInfo(ROLE, storeProvider));
   }
 
@@ -649,6 +665,22 @@ public class QuotaManagerImplTest extends EasyMockTest {
   }
 
   @Test
+  public void testCheckQuotaNewUpdateSkippedForDedicatedDesiredState() {
+    ITaskConfig config = taskConfig(2, 2, 2, false);
+    IJobUpdate update = buildJobUpdate(
+        buildJobUpdateSummaries(UPDATE_KEY).get(0),
+        prodDedicatedTask("dedicatedJob", 1, 1, 1).getAssignedTask().getTask(),
+        1,
+        config,
+        1);
+
+    control.replay();
+
+    QuotaCheckResult checkQuota = quotaManager.checkJobUpdate(update, storeProvider);
+    assertEquals(SUFFICIENT_QUOTA, checkQuota.getResult());
+  }
+
+  @Test
   public void testCheckQuotaNewUpdateSkippedForEmptyDesiredState() {
     ITaskConfig config = taskConfig(2, 2, 2, true);
     IJobUpdate update = buildJobUpdate(
@@ -671,7 +703,7 @@ public class QuotaManagerImplTest extends EasyMockTest {
   public void testSaveQuotaPasses() throws Exception {
     expectNoJobUpdates();
     expectNoCronJobs();
-    IScheduledTask prodTask = createProdTask("foo", 1, 1, 1);
+    IScheduledTask prodTask = prodTask("foo", 1, 1, 1);
     expectTasks(prodTask);
     expectQuota(IResourceAggregate.build(new ResourceAggregate(1, 1, 1)));
 
@@ -706,7 +738,7 @@ public class QuotaManagerImplTest extends EasyMockTest {
   public void testSaveQuotaFailsWhenBelowCurrentReservation() throws Exception {
     expectNoJobUpdates();
     expectNoCronJobs();
-    IScheduledTask prodTask = createProdTask("foo", 10, 100, 100);
+    IScheduledTask prodTask = prodTask("foo", 10, 100, 100);
     expectTasks(prodTask);
     expectQuota(IResourceAggregate.build(new ResourceAggregate(20, 200, 200)));
 
@@ -724,17 +756,17 @@ public class QuotaManagerImplTest extends EasyMockTest {
     expectNoTasks().times(2);
     expectNoJobUpdates().times(2);
 
-    IJobConfiguration job = createJob(createProdTask("pc", 4, 4, 4), 1);
-    expectCronJobs(job, createJob(createNonProdTask("npc", 7, 7, 7), 1)).times(2);
+    IJobConfiguration job = createJob(prodTask("pc", 4, 4, 4), 1);
+    expectCronJobs(job, createJob(nonProdTask("npc", 7, 7, 7), 1)).times(2);
     expectCronJob(job);
 
     control.replay();
 
     QuotaCheckResult checkQuota =
-        quotaManager.checkCronUpdate(createJob(createProdTask("pc", 1, 1, 1), 2), storeProvider);
+        quotaManager.checkCronUpdate(createJob(prodTask("pc", 1, 1, 1), 2), storeProvider);
     assertEquals(SUFFICIENT_QUOTA, checkQuota.getResult());
     assertEquals(
-        new QuotaInfo(from(5, 5, 5), from(4, 4, 4), from(7, 7, 7)),
+        new QuotaInfo(from(5, 5, 5), from(4, 4, 4), EMPTY, from(7, 7, 7), EMPTY),
         quotaManager.getQuotaInfo(ROLE, storeProvider));
   }
 
@@ -744,17 +776,17 @@ public class QuotaManagerImplTest extends EasyMockTest {
     expectNoTasks().times(2);
     expectNoJobUpdates().times(2);
 
-    IJobConfiguration job = createJob(createProdTask("pc", 4, 4, 4), 1);
-    expectCronJobs(job, createJob(createNonProdTask("npc", 7, 7, 7), 1)).times(2);
+    IJobConfiguration job = createJob(prodTask("pc", 4, 4, 4), 1);
+    expectCronJobs(job, createJob(nonProdTask("npc", 7, 7, 7), 1)).times(2);
     expectCronJob(job);
 
     control.replay();
 
     QuotaCheckResult checkQuota =
-        quotaManager.checkCronUpdate(createJob(createProdTask("pc", 5, 5, 5), 1), storeProvider);
+        quotaManager.checkCronUpdate(createJob(prodTask("pc", 5, 5, 5), 1), storeProvider);
     assertEquals(SUFFICIENT_QUOTA, checkQuota.getResult());
     assertEquals(
-        new QuotaInfo(from(5, 5, 5), from(4, 4, 4), from(7, 7, 7)),
+        new QuotaInfo(from(5, 5, 5), from(4, 4, 4), EMPTY, from(7, 7, 7), EMPTY),
         quotaManager.getQuotaInfo(ROLE, storeProvider));
   }
 
@@ -764,17 +796,17 @@ public class QuotaManagerImplTest extends EasyMockTest {
     expectNoTasks().times(2);
     expectNoJobUpdates().times(2);
 
-    IJobConfiguration job = createJob(createProdTask("pc", 4, 4, 4), 1);
+    IJobConfiguration job = createJob(prodTask("pc", 4, 4, 4), 1);
     expectCronJobs(job).times(2);
     expectCronJob(job);
 
     control.replay();
 
     QuotaCheckResult checkQuota =
-        quotaManager.checkCronUpdate(createJob(createProdTask("pc", 5, 5, 5), 2), storeProvider);
+        quotaManager.checkCronUpdate(createJob(prodTask("pc", 5, 5, 5), 2), storeProvider);
     assertEquals(INSUFFICIENT_QUOTA, checkQuota.getResult());
     assertEquals(
-        new QuotaInfo(from(5, 5, 5), from(4, 4, 4), ResourceAggregates.EMPTY),
+        new QuotaInfo(from(5, 5, 5), from(4, 4, 4), EMPTY, EMPTY, EMPTY),
         quotaManager.getQuotaInfo(ROLE, storeProvider));
   }
 
@@ -789,10 +821,10 @@ public class QuotaManagerImplTest extends EasyMockTest {
     control.replay();
 
     QuotaCheckResult checkQuota =
-        quotaManager.checkCronUpdate(createJob(createProdTask("pc", 5, 5, 5), 1), storeProvider);
+        quotaManager.checkCronUpdate(createJob(prodTask("pc", 5, 5, 5), 1), storeProvider);
     assertEquals(SUFFICIENT_QUOTA, checkQuota.getResult());
     assertEquals(
-        new QuotaInfo(from(5, 5, 5), ResourceAggregates.EMPTY, ResourceAggregates.EMPTY),
+        new QuotaInfo(from(5, 5, 5), EMPTY, EMPTY, EMPTY, EMPTY),
         quotaManager.getQuotaInfo(ROLE, storeProvider));
   }
 
@@ -801,7 +833,7 @@ public class QuotaManagerImplTest extends EasyMockTest {
     control.replay();
 
     QuotaCheckResult checkQuota =
-        quotaManager.checkCronUpdate(createJob(createNonProdTask("np", 5, 5, 5), 1), storeProvider);
+        quotaManager.checkCronUpdate(createJob(nonProdTask("np", 5, 5, 5), 1), storeProvider);
 
     assertEquals(SUFFICIENT_QUOTA, checkQuota.getResult());
   }
@@ -903,11 +935,28 @@ public class QuotaManagerImplTest extends EasyMockTest {
         .getTask();
   }
 
-  private IScheduledTask createProdTask(String jobName, int cpus, int ramMb, int diskMb) {
+  private IScheduledTask prodTask(String jobName, int cpus, int ramMb, int diskMb) {
     return createTask(jobName, jobName + "id1", cpus, ramMb, diskMb, true, 0);
   }
 
-  private IScheduledTask createNonProdTask(String jobName, int cpus, int ramMb, int diskMb) {
+  private IScheduledTask prodDedicatedTask(String jobName, int cpus, int ramMb, int diskMb) {
+    return makeDedicated(prodTask(jobName, cpus, ramMb, diskMb));
+  }
+
+  private IScheduledTask nonProdDedicatedTask(String jobName, int cpus, int ramMb, int diskMb) {
+    return makeDedicated(nonProdTask(jobName, cpus, ramMb, diskMb));
+  }
+
+  private static IScheduledTask makeDedicated(IScheduledTask task) {
+    ScheduledTask builder = task.newBuilder();
+    builder.getAssignedTask().getTask().setConstraints(ImmutableSet.of(
+        new Constraint(
+            "dedicated",
+            TaskConstraint.value(new ValueConstraint(false, ImmutableSet.of("host"))))));
+    return IScheduledTask.build(builder);
+  }
+
+  private IScheduledTask nonProdTask(String jobName, int cpus, int ramMb, int diskMb) {
     return createTask(jobName, jobName + "id1", cpus, ramMb, diskMb, false, 0);
   }
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/01e611ad/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 a8df446..6e032a6 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
@@ -88,7 +88,7 @@ public class SnapshotStoreImplTest extends EasyMockTest {
         IScheduledTask.build(new ScheduledTask().setStatus(ScheduleStatus.PENDING)));
     Set<QuotaConfiguration> quotas =
         ImmutableSet.of(
-            new QuotaConfiguration("steve", ResourceAggregates.none().newBuilder()));
+            new QuotaConfiguration("steve", ResourceAggregates.EMPTY.newBuilder()));
     IHostAttributes attribute = IHostAttributes.build(
         new HostAttributes("host", ImmutableSet.of(new Attribute("attr", ImmutableSet.of("value"))))
             .setSlaveId("slave id"));
@@ -122,7 +122,7 @@ public class SnapshotStoreImplTest extends EasyMockTest {
     storageUtil.expectOperations();
     expect(storageUtil.taskStore.fetchTasks(Query.unscoped())).andReturn(tasks);
     expect(storageUtil.quotaStore.fetchQuotas())
-        .andReturn(ImmutableMap.of("steve", ResourceAggregates.none()));
+        .andReturn(ImmutableMap.of("steve", ResourceAggregates.EMPTY));
     expect(storageUtil.attributeStore.getHostAttributes())
         .andReturn(ImmutableSet.of(attribute, legacyAttribute));
     expect(storageUtil.jobStore.fetchJobs())
@@ -137,7 +137,7 @@ public class SnapshotStoreImplTest extends EasyMockTest {
 
     expectDataWipe();
     storageUtil.taskStore.saveTasks(tasks);
-    storageUtil.quotaStore.saveQuota("steve", ResourceAggregates.none());
+    storageUtil.quotaStore.saveQuota("steve", ResourceAggregates.EMPTY);
     expect(storageUtil.attributeStore.saveHostAttributes(attribute)).andReturn(true);
     storageUtil.jobStore.saveAcceptedJob(IJobConfiguration.build(job.getJobConfiguration()));
     storageUtil.schedulerStore.saveFrameworkId(frameworkId);

http://git-wip-us.apache.org/repos/asf/aurora/blob/01e611ad/src/test/java/org/apache/aurora/scheduler/thrift/Fixtures.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/thrift/Fixtures.java b/src/test/java/org/apache/aurora/scheduler/thrift/Fixtures.java
index e0110e7..b058719 100644
--- a/src/test/java/org/apache/aurora/scheduler/thrift/Fixtures.java
+++ b/src/test/java/org/apache/aurora/scheduler/thrift/Fixtures.java
@@ -42,7 +42,6 @@ import org.apache.aurora.gen.ResponseCode;
 import org.apache.aurora.gen.ResponseDetail;
 import org.apache.aurora.gen.Result;
 import org.apache.aurora.gen.ScheduledTask;
-import org.apache.aurora.gen.ServerInfo;
 import org.apache.aurora.gen.TaskConfig;
 import org.apache.aurora.scheduler.base.JobKeys;
 import org.apache.aurora.scheduler.quota.QuotaCheckResult;
@@ -52,10 +51,8 @@ import org.apache.aurora.scheduler.storage.entities.ILock;
 import org.apache.aurora.scheduler.storage.entities.ILockKey;
 import org.apache.aurora.scheduler.storage.entities.IResourceAggregate;
 import org.apache.aurora.scheduler.storage.entities.IScheduledTask;
-import org.apache.aurora.scheduler.storage.entities.IServerInfo;
 
 import static org.apache.aurora.gen.ResponseCode.OK;
-import static org.apache.aurora.gen.apiConstants.THRIFT_API_VERSION;
 import static org.apache.aurora.scheduler.quota.QuotaCheckResult.Result.INSUFFICIENT_QUOTA;
 import static org.apache.aurora.scheduler.quota.QuotaCheckResult.Result.SUFFICIENT_QUOTA;
 import static org.junit.Assert.assertEquals;
@@ -75,11 +72,6 @@ final class Fixtures {
   static final IJobUpdateKey UPDATE_KEY =
       IJobUpdateKey.build(new JobUpdateKey(JOB_KEY.newBuilder(), UPDATE_ID));
   static final UUID UU_ID = UUID.fromString(UPDATE_ID);
-  static final IServerInfo SERVER_INFO =
-      IServerInfo.build(new ServerInfo()
-          .setClusterName("test")
-          .setThriftAPIVersion(THRIFT_API_VERSION)
-          .setStatsUrlPrefix("fake_url"));
   private static final Function<String, ResponseDetail> MESSAGE_TO_DETAIL =
       new Function<String, ResponseDetail>() {
         @Override
@@ -90,8 +82,6 @@ final class Fixtures {
   static final String CRON_SCHEDULE = "0 * * * *";
   static final IResourceAggregate QUOTA =
       IResourceAggregate.build(new ResourceAggregate(10.0, 1024, 2048));
-  static final IResourceAggregate CONSUMED =
-      IResourceAggregate.build(new ResourceAggregate(0.0, 0, 0));
   static final QuotaCheckResult ENOUGH_QUOTA = new QuotaCheckResult(SUFFICIENT_QUOTA);
   static final QuotaCheckResult NOT_ENOUGH_QUOTA = new QuotaCheckResult(INSUFFICIENT_QUOTA);
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/01e611ad/src/test/java/org/apache/aurora/scheduler/thrift/ReadOnlySchedulerImplTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/thrift/ReadOnlySchedulerImplTest.java b/src/test/java/org/apache/aurora/scheduler/thrift/ReadOnlySchedulerImplTest.java
index 4d4e752..37c8129 100644
--- a/src/test/java/org/apache/aurora/scheduler/thrift/ReadOnlySchedulerImplTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/thrift/ReadOnlySchedulerImplTest.java
@@ -33,6 +33,7 @@ import org.apache.aurora.gen.AssignedTask;
 import org.apache.aurora.gen.ConfigGroup;
 import org.apache.aurora.gen.ConfigSummary;
 import org.apache.aurora.gen.ConfigSummaryResult;
+import org.apache.aurora.gen.GetQuotaResult;
 import org.apache.aurora.gen.Identity;
 import org.apache.aurora.gen.JobConfiguration;
 import org.apache.aurora.gen.JobKey;
@@ -46,7 +47,6 @@ import org.apache.aurora.gen.JobUpdateSummary;
 import org.apache.aurora.gen.PendingReason;
 import org.apache.aurora.gen.PopulateJobResult;
 import org.apache.aurora.gen.ReadOnlyScheduler;
-import org.apache.aurora.gen.ResourceAggregate;
 import org.apache.aurora.gen.Response;
 import org.apache.aurora.gen.Result;
 import org.apache.aurora.gen.RoleSummary;
@@ -73,7 +73,6 @@ import org.apache.aurora.scheduler.storage.entities.IJobKey;
 import org.apache.aurora.scheduler.storage.entities.IJobUpdateDetails;
 import org.apache.aurora.scheduler.storage.entities.IJobUpdateQuery;
 import org.apache.aurora.scheduler.storage.entities.IJobUpdateSummary;
-import org.apache.aurora.scheduler.storage.entities.IResourceAggregate;
 import org.apache.aurora.scheduler.storage.entities.IScheduledTask;
 import org.apache.aurora.scheduler.storage.entities.ITaskConfig;
 import org.apache.aurora.scheduler.storage.testing.StorageTestUtil;
@@ -81,7 +80,10 @@ import org.junit.Before;
 import org.junit.Test;
 
 import static org.apache.aurora.gen.ResponseCode.INVALID_REQUEST;
-import static org.apache.aurora.scheduler.thrift.Fixtures.CONSUMED;
+import static org.apache.aurora.scheduler.ResourceAggregates.LARGE;
+import static org.apache.aurora.scheduler.ResourceAggregates.MEDIUM;
+import static org.apache.aurora.scheduler.ResourceAggregates.SMALL;
+import static org.apache.aurora.scheduler.ResourceAggregates.XLARGE;
 import static org.apache.aurora.scheduler.thrift.Fixtures.CRON_SCHEDULE;
 import static org.apache.aurora.scheduler.thrift.Fixtures.JOB_KEY;
 import static org.apache.aurora.scheduler.thrift.Fixtures.LOCK;
@@ -297,19 +299,21 @@ public class ReadOnlySchedulerImplTest extends EasyMockTest {
     QuotaInfo infoMock = createMock(QuotaInfo.class);
     expect(quotaManager.getQuotaInfo(ROLE, storageUtil.storeProvider)).andReturn(infoMock);
     expect(infoMock.getQuota()).andReturn(QUOTA);
-    expect(infoMock.getProdConsumption()).andReturn(CONSUMED);
-    IResourceAggregate nonProdConsumed = IResourceAggregate.build(new ResourceAggregate(1, 0, 0));
-    expect(infoMock.getNonProdConsumption()).andReturn(nonProdConsumed);
+    expect(infoMock.getProdSharedConsumption()).andReturn(XLARGE);
+    expect(infoMock.getProdDedicatedConsumption()).andReturn(LARGE);
+    expect(infoMock.getNonProdSharedConsumption()).andReturn(MEDIUM);
+    expect(infoMock.getNonProdDedicatedConsumption()).andReturn(SMALL);
     control.replay();
 
+    GetQuotaResult expected = new GetQuotaResult()
+        .setQuota(QUOTA.newBuilder())
+        .setProdSharedConsumption(XLARGE.newBuilder())
+        .setProdDedicatedConsumption(LARGE.newBuilder())
+        .setNonProdSharedConsumption(MEDIUM.newBuilder())
+        .setNonProdDedicatedConsumption(SMALL.newBuilder());
+
     Response response = assertOkResponse(thrift.getQuota(ROLE));
-    assertEquals(QUOTA.newBuilder(), response.getResult().getGetQuotaResult().getQuota());
-    assertEquals(
-        CONSUMED.newBuilder(),
-        response.getResult().getGetQuotaResult().getProdConsumption());
-    assertEquals(
-        nonProdConsumed.newBuilder(),
-        response.getResult().getGetQuotaResult().getNonProdConsumption());
+    assertEquals(expected, response.getResult().getGetQuotaResult());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/aurora/blob/01e611ad/src/test/python/apache/aurora/admin/test_admin.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/admin/test_admin.py b/src/test/python/apache/aurora/admin/test_admin.py
index d793293..8e204ab 100644
--- a/src/test/python/apache/aurora/admin/test_admin.py
+++ b/src/test/python/apache/aurora/admin/test_admin.py
@@ -130,7 +130,7 @@ class TestIncreaseQuotaCommand(AuroraClientCommandTest):
         responseCode=response_code,
         details=[ResponseDetail(message='test')],
         result=Result(getQuotaResult=GetQuotaResult(
-            quota=quota, prodConsumption=prod, nonProdConsumption=non_prod))
+            quota=quota, prodSharedConsumption=prod, nonProdSharedConsumption=non_prod))
     )
 
   def test_increase_quota(self):
@@ -167,7 +167,7 @@ class TestSetQuotaCommand(AuroraClientCommandTest):
     response_code = ResponseCode.OK if response_code is None else response_code
     resp = Response(responseCode=response_code, details=[ResponseDetail(message='test')])
     resp.result = Result(getQuotaResult=GetQuotaResult(
-      quota=quota, prodConsumption=prod, nonProdConsumption=non_prod))
+      quota=quota, prodSharedConsumption=prod, nonProdSharedConsumption=non_prod))
     return resp
 
   def test_set_quota(self):

http://git-wip-us.apache.org/repos/asf/aurora/blob/01e611ad/src/test/python/apache/aurora/client/api/test_quota_check.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/api/test_quota_check.py b/src/test/python/apache/aurora/client/api/test_quota_check.py
index 6c9bc37..7035bb5 100644
--- a/src/test/python/apache/aurora/client/api/test_quota_check.py
+++ b/src/test/python/apache/aurora/client/api/test_quota_check.py
@@ -47,7 +47,7 @@ class QuotaCheckTest(unittest.TestCase):
     resp = Response(responseCode=response_code, details=[ResponseDetail(message='test')])
     resp.result = Result(
         getQuotaResult=GetQuotaResult(
-          quota=deepcopy(allocated), prodConsumption=deepcopy(consumed)))
+          quota=deepcopy(allocated), prodSharedConsumption=deepcopy(consumed)))
     self._scheduler.getQuota.return_value = resp
 
   def assert_result(self, prod, released, acquired, expected_code=None):

http://git-wip-us.apache.org/repos/asf/aurora/blob/01e611ad/src/test/python/apache/aurora/client/cli/test_quota.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/test_quota.py b/src/test/python/apache/aurora/client/cli/test_quota.py
index 3573e4c..bf79180 100644
--- a/src/test/python/apache/aurora/client/cli/test_quota.py
+++ b/src/test/python/apache/aurora/client/cli/test_quota.py
@@ -30,8 +30,8 @@ class TestGetQuotaCommand(AuroraClientCommandTest):
     response = cls.create_simple_success_response()
     response.result = Result(getQuotaResult=GetQuotaResult(
         quota=ResourceAggregate(numCpus=5, ramMb=20480, diskMb=40960),
-        prodConsumption=None,
-        nonProdConsumption=None
+        prodSharedConsumption=None,
+        nonProdSharedConsumption=None
     ))
     api.get_quota.return_value = response
 
@@ -41,8 +41,8 @@ class TestGetQuotaCommand(AuroraClientCommandTest):
     response = cls.create_simple_success_response()
     response.result = Result(getQuotaResult=GetQuotaResult(
       quota=ResourceAggregate(numCpus=5, ramMb=20480, diskMb=40960),
-      prodConsumption=ResourceAggregate(numCpus=1, ramMb=1024, diskMb=2048),
-      nonProdConsumption=ResourceAggregate(numCpus=1, ramMb=1024, diskMb=2048),
+      prodSharedConsumption=ResourceAggregate(numCpus=1, ramMb=1024, diskMb=2048),
+      nonProdSharedConsumption=ResourceAggregate(numCpus=1, ramMb=1024, diskMb=2048),
     ))
     api.get_quota.return_value = response
 
@@ -63,9 +63,10 @@ class TestGetQuotaCommand(AuroraClientCommandTest):
             json.loads(self._get_quota(False, ['quota', 'get', '--write-json', 'west/bozo'])))
 
   def test_get_quota_with_consumption_json(self):
-    expected_response = json.loads('{"quota":{"numCpus":5,"ramMb":20480,"diskMb":40960},'
-                                   '"prodConsumption":{"numCpus":1,"ramMb":1024,"diskMb":2048},'
-                                   '"nonProdConsumption":{"numCpus":1,"ramMb":1024,"diskMb":2048}}')
+    expected_response = json.loads(
+        '{"quota":{"numCpus":5,"ramMb":20480,"diskMb":40960},'
+        '"prodSharedConsumption":{"numCpus":1,"ramMb":1024,"diskMb":2048},'
+        '"nonProdSharedConsumption":{"numCpus":1,"ramMb":1024,"diskMb":2048}}')
     assert (expected_response ==
             json.loads(self._get_quota(True, ['quota', 'get', '--write-json', 'west/bozo'])))