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 2016/04/11 23:45:50 UTC

[1/2] aurora git commit: Moving resource-related classes into a new package

Repository: aurora
Updated Branches:
  refs/heads/master bafdd71a0 -> 46ce98d8c


http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/test/java/org/apache/aurora/scheduler/AcceptedOfferTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/AcceptedOfferTest.java b/src/test/java/org/apache/aurora/scheduler/AcceptedOfferTest.java
deleted file mode 100644
index 49ced12..0000000
--- a/src/test/java/org/apache/aurora/scheduler/AcceptedOfferTest.java
+++ /dev/null
@@ -1,304 +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;
-
-import java.util.Collections;
-import java.util.List;
-import java.util.Set;
-
-import com.google.common.base.Optional;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-
-import org.apache.aurora.common.quantity.Amount;
-import org.apache.aurora.common.quantity.Data;
-import org.apache.aurora.scheduler.base.TaskTestUtil;
-import org.apache.mesos.Protos;
-import org.apache.mesos.Protos.Resource;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-public class AcceptedOfferTest {
-
-  private static final Optional<String> TEST_ROLE = Optional.of("test-role");
-  private static final Optional<String> ABSENT_ROLE = Optional.absent();
-  private static final ResourceSlot TASK_SLOT = new ResourceSlot(
-      4, Amount.of(100L, Data.MB), Amount.of(200L, Data.MB), 0);
-  private static final ResourceSlot EXECUTOR_SLOT = new ResourceSlot(
-      0.25, Amount.of(25L, Data.MB), Amount.of(75L, Data.MB), 0);
-  private static final ResourceSlot TOTAL_SLOT = EXECUTOR_SLOT.add(TASK_SLOT);
-  private static final Integer[] TASK_PORTS = {80, 90};
-  private static final Set<Integer> TASK_PORTS_SET = ImmutableSet.copyOf(TASK_PORTS);
-
-  @Test
-  public void testReservedPredicates() {
-    Protos.Resource withRole = makeScalar(ResourceType.CPUS.getName(), TEST_ROLE, false, 1.0);
-    assertTrue(AcceptedOffer.RESERVED.apply(withRole));
-    assertFalse(AcceptedOffer.NOT_RESERVED.apply(withRole));
-    Protos.Resource absentRole = makeScalar(ResourceType.CPUS.getName(), ABSENT_ROLE, false, 1.0);
-    assertFalse(AcceptedOffer.RESERVED.apply(absentRole));
-    assertTrue(AcceptedOffer.NOT_RESERVED.apply(absentRole));
-  }
-
-  @Test
-  public void testAllocateEmpty() {
-    AcceptedOffer acceptedOffer = AcceptedOffer.create(
-        fakeOffer(Collections.emptyList()),
-        ResourceSlot.NONE,
-        ResourceSlot.NONE,
-        ImmutableSet.of(),
-        TaskTestUtil.DEV_TIER);
-    assertEquals(Collections.emptyList(), acceptedOffer.getTaskResources());
-    assertEquals(Collections.emptyList(), acceptedOffer.getExecutorResources());
-  }
-
-  @Test
-  public void testAllocateRange() {
-    List<Resource> resources = ImmutableList.<Resource>builder()
-        .add(makePortResource(Optional.absent(), 80, 81, 90, 91, 92, 93))
-        .add(makePortResource(TEST_ROLE, 100, 101))
-        .build();
-    AcceptedOffer acceptedOffer = AcceptedOffer.create(
-        fakeOffer(resources),
-        ResourceSlot.NONE,
-        ResourceSlot.NONE,
-        ImmutableSet.of(80, 90, 100),
-        TaskTestUtil.DEV_TIER);
-
-    List<Resource> expected = ImmutableList.<Resource>builder()
-        // Because we prefer reserved resources and handle them before non-reserved resources,
-        // result should have ports for the reserved resources first.
-        .add(makePortResource(TEST_ROLE, 100))
-        .add(makePortResource(Optional.absent(), 80, 90))
-        .build();
-    assertEquals(expected, acceptedOffer.getTaskResources());
-    assertEquals(Collections.emptyList(), acceptedOffer.getExecutorResources());
-  }
-
-  @Test(expected = Resources.InsufficientResourcesException.class)
-  public void testAllocateRangeInsufficent() {
-    List<Resource> resources = ImmutableList.of(
-        makePortResource(ABSENT_ROLE, 80),
-        makePortResource(ABSENT_ROLE, 100, 101));
-    AcceptedOffer.create(
-        fakeOffer(resources),
-        ResourceSlot.NONE,
-        ResourceSlot.NONE,
-        ImmutableSet.of(80, 90, 100),
-        TaskTestUtil.DEV_TIER);
-  }
-
-  @Test
-  public void testAllocateSingleRole() {
-    runAllocateSingleRole(ABSENT_ROLE, false);
-    runAllocateSingleRole(ABSENT_ROLE, true);
-    runAllocateSingleRole(TEST_ROLE, false);
-    runAllocateSingleRole(TEST_ROLE, true);
-  }
-
-  private void runAllocateSingleRole(Optional<String> role, boolean cpuRevocable) {
-    List<Resource> resources = ImmutableList.<Resource>builder()
-        .add(makeScalar(
-            ResourceType.CPUS.getName(), role, cpuRevocable, TOTAL_SLOT.getNumCpus()))
-        .add(makeScalar(
-            ResourceType.RAM_MB.getName(), role, false, TOTAL_SLOT.getRam().as(Data.MB)))
-        .add(makeScalar(
-            ResourceType.DISK_MB.getName(), role, false, TOTAL_SLOT.getDisk().as(Data.MB)))
-        .add(makePortResource(role, TASK_PORTS))
-        .build();
-    Protos.Offer offer = fakeOffer(resources);
-
-    AcceptedOffer offerAllocation = AcceptedOffer.create(
-        offer, TASK_SLOT, EXECUTOR_SLOT, TASK_PORTS_SET, new TierInfo(false, cpuRevocable));
-
-    List<Resource> taskList = ImmutableList.<Resource>builder()
-        .add(makeScalar(ResourceType.CPUS.getName(), role, cpuRevocable, TASK_SLOT.getNumCpus()))
-        .add(makeScalar(ResourceType.RAM_MB.getName(), role, false, TASK_SLOT.getRam().as(Data.MB)))
-        .add(makeScalar(
-            ResourceType.DISK_MB.getName(), role, false, TASK_SLOT.getDisk().as(Data.MB)))
-        .add(makePortResource(role, TASK_PORTS))
-        .build();
-    assertEquals(taskList, offerAllocation.getTaskResources());
-
-    List<Resource> executorList = ImmutableList.<Resource>builder()
-        .add(makeScalar(
-            ResourceType.CPUS.getName(), role, cpuRevocable, EXECUTOR_SLOT.getNumCpus()))
-        .add(makeScalar(
-            ResourceType.RAM_MB.getName(), role, false, EXECUTOR_SLOT.getRam().as(Data.MB)))
-        .add(makeScalar(
-            ResourceType.DISK_MB.getName(), role, false, EXECUTOR_SLOT.getDisk().as(Data.MB)))
-        .build();
-    assertEquals(executorList, offerAllocation.getExecutorResources());
-  }
-
-  @Test(expected = Resources.InsufficientResourcesException.class)
-  public void testAllocateSingleRoleInsufficient() {
-    List<Resource> resources = ImmutableList.<Resource>builder()
-        // EXECUTOR_SLOT's CPU is not included here.
-        .add(makeScalar(ResourceType.CPUS.getName(), TEST_ROLE, false, TASK_SLOT.getNumCpus()))
-        .add(makeScalar(
-            ResourceType.RAM_MB.getName(), TEST_ROLE, false, TOTAL_SLOT.getRam().as(Data.MB)))
-        .add(makeScalar(
-            ResourceType.DISK_MB.getName(), TEST_ROLE, false, TOTAL_SLOT.getDisk().as(Data.MB)))
-        .add(makePortResource(TEST_ROLE, TASK_PORTS))
-        .build();
-    Protos.Offer offer = fakeOffer(resources);
-
-    AcceptedOffer.create(
-        offer, TASK_SLOT, EXECUTOR_SLOT, TASK_PORTS_SET, new TierInfo(false, false));
-  }
-
-  @Test
-  public void testMultipleRoles() {
-    runMultipleRoles(false);
-    runMultipleRoles(true);
-  }
-
-  private void runMultipleRoles(boolean cpuRevocable) {
-    List<Resource> resources = ImmutableList.<Resource>builder()
-        // Make cpus come from two roles.
-        .add(makeScalar(
-            ResourceType.CPUS.getName(),
-            TEST_ROLE,
-            cpuRevocable,
-            EXECUTOR_SLOT.getNumCpus()))
-        .add(makeScalar(
-            ResourceType.CPUS.getName(),
-            ABSENT_ROLE,
-            cpuRevocable,
-            TASK_SLOT.getNumCpus()))
-        // Make ram come from default role
-        .add(makeScalar(
-            ResourceType.RAM_MB.getName(),
-            ABSENT_ROLE,
-            false,
-            TOTAL_SLOT.getRam().as(Data.MB)))
-        // Make disk come from non-default role.
-        .add(makeScalar(
-            ResourceType.DISK_MB.getName(),
-            TEST_ROLE,
-            false,
-            TOTAL_SLOT.getDisk().as(Data.MB)))
-        .add(makePortResource(TEST_ROLE, 80))
-        .add(makePortResource(ABSENT_ROLE, 90))
-        .build();
-
-    Protos.Offer offer = fakeOffer(resources);
-
-    AcceptedOffer offerAllocation = AcceptedOffer.create(
-        offer, TASK_SLOT, EXECUTOR_SLOT, TASK_PORTS_SET, new TierInfo(false, cpuRevocable));
-
-    List<Resource> taskList = ImmutableList.<Resource>builder()
-        // We intentionally sliced the offer resource to not align with TASK_SLOT's num cpus.
-        .add(makeScalar(
-            ResourceType.CPUS.getName(), TEST_ROLE, cpuRevocable, EXECUTOR_SLOT.getNumCpus()))
-        .add(makeScalar(
-            ResourceType.CPUS.getName(),
-            ABSENT_ROLE,
-            cpuRevocable,
-            TASK_SLOT.subtract(EXECUTOR_SLOT).getNumCpus()))
-        .add(makeScalar(
-            ResourceType.RAM_MB.getName(), ABSENT_ROLE, false, TASK_SLOT.getRam().as(Data.MB)))
-        .add(makeScalar(
-            ResourceType.DISK_MB.getName(), TEST_ROLE, false, TASK_SLOT.getDisk().as(Data.MB)))
-        .add(makePortResource(TEST_ROLE, 80))
-        .add(makePortResource(ABSENT_ROLE, 90))
-        .build();
-    assertEquals(taskList, offerAllocation.getTaskResources());
-
-    List<Resource> executorList = ImmutableList.<Resource>builder()
-        .add(makeScalar(
-            ResourceType.CPUS.getName(), ABSENT_ROLE, cpuRevocable, EXECUTOR_SLOT.getNumCpus()))
-        .add(makeScalar(
-            ResourceType.RAM_MB.getName(), ABSENT_ROLE, false, EXECUTOR_SLOT.getRam().as(Data.MB)))
-        .add(makeScalar(
-            ResourceType.DISK_MB.getName(), TEST_ROLE, false, EXECUTOR_SLOT.getDisk().as(Data.MB)))
-        .build();
-    assertEquals(executorList, offerAllocation.getExecutorResources());
-  }
-
-  @Test(expected = Resources.InsufficientResourcesException.class)
-  public void testMultipleRolesInsufficient() {
-    // Similar to testMultipleRoles, but make some of cpus as revocable
-    List<Resource> resources = ImmutableList.<Resource>builder()
-        // Make cpus come from two roles.
-        .add(makeScalar(
-            ResourceType.CPUS.getName(),
-            TEST_ROLE,
-            true,
-            EXECUTOR_SLOT.getNumCpus()))
-        .add(makeScalar(
-            ResourceType.CPUS.getName(),
-            ABSENT_ROLE,
-            false,
-            TASK_SLOT.getNumCpus()))
-        // Make ram come from default role
-        .add(makeScalar(
-            ResourceType.RAM_MB.getName(),
-            ABSENT_ROLE,
-            false,
-            TOTAL_SLOT.getRam().as(Data.MB)))
-        // Make disk come from non-default role.
-        .add(makeScalar(
-            ResourceType.DISK_MB.getName(),
-            TEST_ROLE,
-            false,
-            TOTAL_SLOT.getDisk().as(Data.MB)))
-        .add(makePortResource(TEST_ROLE, 80))
-        .add(makePortResource(ABSENT_ROLE, 90))
-        .build();
-    Protos.Offer offer = fakeOffer(resources);
-    // We don't have enough resource to satisfy a non-revocable request.
-    AcceptedOffer.create(
-        offer, TASK_SLOT, EXECUTOR_SLOT, TASK_PORTS_SET, new TierInfo(false, false));
-  }
-
-  private static Resource makePortResource(Optional<String> role, Integer... values) {
-    Resource.Builder prototype = Resource.newBuilder()
-        .setType(Protos.Value.Type.RANGES)
-        .setName(ResourceType.PORTS.getName());
-    if (role.isPresent()) {
-      prototype.setRole(role.get());
-    }
-    return AcceptedOffer.makeMesosRangeResource(prototype.build(), ImmutableSet.copyOf(values));
-  }
-
-  private static Resource makeScalar(
-      String name, Optional<String> role, boolean revocable, double value) {
-    Resource.Builder resource = Resource.newBuilder()
-        .setName(name)
-        .setType(Protos.Value.Type.SCALAR)
-        .setScalar(Protos.Value.Scalar.newBuilder().setValue(value));
-    if (role.isPresent()) {
-      resource.setRole(role.get());
-    }
-    if (revocable) {
-      resource.setRevocable(Resource.RevocableInfo.getDefaultInstance());
-    }
-    return resource.build();
-  }
-
-  private static Protos.Offer fakeOffer(List<Resource> resources) {
-    return Protos.Offer.newBuilder()
-        .setId(Protos.OfferID.newBuilder().setValue("offer-id"))
-        .setFrameworkId(Protos.FrameworkID.newBuilder().setValue("framework-id"))
-        .setSlaveId(Protos.SlaveID.newBuilder().setValue("slave-id"))
-        .setHostname("hostname")
-        .addAllResources(resources)
-        .build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/test/java/org/apache/aurora/scheduler/ResourceSlotTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/ResourceSlotTest.java b/src/test/java/org/apache/aurora/scheduler/ResourceSlotTest.java
deleted file mode 100644
index 6564e82..0000000
--- a/src/test/java/org/apache/aurora/scheduler/ResourceSlotTest.java
+++ /dev/null
@@ -1,170 +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;
-
-import java.util.Set;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
-
-import org.apache.aurora.common.collections.Pair;
-import org.apache.aurora.common.quantity.Amount;
-import org.apache.aurora.common.quantity.Data;
-import org.apache.aurora.gen.TaskConfig;
-import org.apache.aurora.scheduler.storage.entities.ITaskConfig;
-import org.apache.mesos.Protos;
-import org.junit.Test;
-
-import static org.apache.aurora.scheduler.ResourceSlot.makeMesosRangeResource;
-import static org.apache.aurora.scheduler.ResourceSlot.makeMesosResource;
-import static org.apache.aurora.scheduler.ResourceSlot.sum;
-import static org.apache.aurora.scheduler.ResourceType.CPUS;
-import static org.apache.aurora.scheduler.ResourceType.DISK_MB;
-import static org.apache.aurora.scheduler.ResourceType.PORTS;
-import static org.apache.aurora.scheduler.ResourceType.RAM_MB;
-import static org.apache.aurora.scheduler.base.TaskTestUtil.DEV_TIER;
-import static org.apache.aurora.scheduler.base.TaskTestUtil.REVOCABLE_TIER;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-
-public class ResourceSlotTest {
-
-  private static final ResourceSlot NEGATIVE_ONE =
-      new ResourceSlot(-1.0, Amount.of(-1L, Data.MB), Amount.of(-1L, Data.MB), -1);
-  private static final ResourceSlot ONE =
-      new ResourceSlot(1.0, Amount.of(1L, Data.MB), Amount.of(1L, Data.MB), 1);
-  private static final ResourceSlot TWO =
-      new ResourceSlot(2.0, Amount.of(2L, Data.MB), Amount.of(2L, Data.MB), 2);
-  private static final ResourceSlot THREE =
-      new ResourceSlot(3.0, Amount.of(3L, Data.MB), Amount.of(3L, Data.MB), 3);
-  private static final ITaskConfig TASK = ITaskConfig.build(new TaskConfig()
-      .setNumCpus(1.0)
-      .setRamMb(1024)
-      .setDiskMb(2048)
-      .setRequestedPorts(ImmutableSet.of("http", "debug")));
-
-  @Test
-  public void testMaxElements() {
-    ResourceSlot highRAM = new ResourceSlot(1, Amount.of(8L, Data.GB), Amount.of(10L, Data.MB), 0);
-    ResourceSlot rest = new ResourceSlot(10, Amount.of(1L, Data.MB), Amount.of(10L, Data.GB), 1);
-
-    ResourceSlot result = ResourceSlot.maxElements(highRAM, rest);
-    assertEquals(result.getNumCpus(), 10, 0.001);
-    assertEquals(result.getRam(), Amount.of(8L, Data.GB));
-    assertEquals(result.getDisk(), Amount.of(10L, Data.GB));
-    assertEquals(result.getNumPorts(), 1);
-  }
-
-  @Test
-  public void testSubtract() {
-    assertEquals(ONE, TWO.subtract(ONE));
-    assertEquals(TWO, THREE.subtract(ONE));
-    assertEquals(NEGATIVE_ONE, ONE.subtract(TWO));
-    assertEquals(NEGATIVE_ONE, TWO.subtract(THREE));
-  }
-
-  @Test
-  public void testAdd() {
-    assertEquals(TWO, ONE.add(ONE));
-    assertEquals(THREE, ONE.add(TWO));
-    assertEquals(THREE, TWO.add(ONE));
-  }
-
-  @Test
-  public void testSum() {
-    assertEquals(THREE, sum(ImmutableList.of(ONE, ONE, ONE)));
-  }
-
-  @Test
-  public void testToResourceListNoRevoca() {
-    ResourceSlot resources = ResourceSlot.from(TASK);
-    assertEquals(
-        ImmutableSet.of(
-            makeMesosResource(CPUS, TASK.getNumCpus(), false),
-            makeMesosResource(RAM_MB, TASK.getRamMb(), false),
-            makeMesosResource(DISK_MB, TASK.getDiskMb(), false)),
-        ImmutableSet.copyOf(resources.toResourceList(DEV_TIER)));
-  }
-
-  @Test
-  public void testToResourceListRevocable() {
-    ResourceSlot resources = ResourceSlot.from(TASK);
-    assertEquals(
-        ImmutableSet.of(
-            makeMesosResource(CPUS, TASK.getNumCpus(), true),
-            makeMesosResource(RAM_MB, TASK.getRamMb(), false),
-            makeMesosResource(DISK_MB, TASK.getDiskMb(), false)),
-        ImmutableSet.copyOf(resources.toResourceList(REVOCABLE_TIER)));
-  }
-
-  @Test
-  public void testToResourceListNoPorts() {
-    ResourceSlot resources = ResourceSlot.from(TASK);
-    assertEquals(
-        ImmutableSet.of(
-            makeMesosResource(CPUS, TASK.getNumCpus(), true),
-            makeMesosResource(RAM_MB, TASK.getRamMb(), false),
-            makeMesosResource(DISK_MB, TASK.getDiskMb(), false)),
-        ImmutableSet.copyOf(resources.toResourceList(REVOCABLE_TIER)));
-  }
-
-  @Test
-  public void testRangeResourceEmpty() {
-    expectRanges(ImmutableSet.of(), ImmutableSet.of());
-  }
-
-  @Test
-  public void testRangeResourceOneEntry() {
-    expectRanges(ImmutableSet.of(Pair.of(5L, 5L)), ImmutableSet.of(5));
-    expectRanges(ImmutableSet.of(Pair.of(0L, 0L)), ImmutableSet.of(0));
-  }
-
-  @Test
-  public void testRangeResourceNonContiguous() {
-    expectRanges(ImmutableSet.of(Pair.of(1L, 1L), Pair.of(3L, 3L), Pair.of(5L, 5L)),
-        ImmutableSet.of(5, 1, 3));
-  }
-
-  @Test
-  public void testRangeResourceContiguous() {
-    expectRanges(ImmutableSet.of(Pair.of(1L, 2L), Pair.of(4L, 5L), Pair.of(7L, 9L)),
-        ImmutableSet.of(8, 2, 4, 5, 7, 9, 1));
-  }
-
-  @Test
-  public void testEqualsBadType() {
-    ResourceSlot resources = ResourceSlot.from(TASK);
-    assertNotEquals(resources, "Hello");
-    assertNotEquals(resources, null);
-  }
-
-  @Test
-  public void testOrder() {
-    assertEquals(
-        ImmutableList.of(ONE, TWO, THREE, THREE),
-        ResourceSlot.ORDER.sortedCopy(ImmutableList.of(THREE, ONE, TWO, THREE)));
-  }
-
-  private void expectRanges(Set<Pair<Long, Long>> expected, Set<Integer> values) {
-    Protos.Resource resource = makeMesosRangeResource(PORTS, values);
-    assertEquals(Protos.Value.Type.RANGES, resource.getType());
-    assertEquals(PORTS.getName(), resource.getName());
-
-    Set<Pair<Long, Long>> actual = ImmutableSet.copyOf(Iterables.transform(
-        resource.getRanges().getRangeList(),
-        range -> Pair.of(range.getBegin(), range.getEnd())));
-    assertEquals(expected, actual);
-  }
-}

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/test/java/org/apache/aurora/scheduler/ResourcesTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/ResourcesTest.java b/src/test/java/org/apache/aurora/scheduler/ResourcesTest.java
deleted file mode 100644
index ea74842..0000000
--- a/src/test/java/org/apache/aurora/scheduler/ResourcesTest.java
+++ /dev/null
@@ -1,176 +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;
-
-import java.util.Set;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-
-import org.apache.aurora.common.collections.Pair;
-import org.apache.aurora.common.quantity.Amount;
-import org.apache.aurora.scheduler.Resources.InsufficientResourcesException;
-import org.apache.mesos.Protos;
-import org.apache.mesos.Protos.Resource;
-import org.apache.mesos.Protos.Value.Range;
-import org.apache.mesos.Protos.Value.Ranges;
-import org.junit.Test;
-
-import static org.apache.aurora.common.quantity.Data.MB;
-import static org.apache.aurora.scheduler.ResourceSlot.makeMesosResource;
-import static org.apache.aurora.scheduler.ResourceType.CPUS;
-import static org.apache.aurora.scheduler.ResourceType.DISK_MB;
-import static org.apache.aurora.scheduler.ResourceType.PORTS;
-import static org.apache.aurora.scheduler.ResourceType.RAM_MB;
-import static org.apache.aurora.scheduler.base.TaskTestUtil.DEV_TIER;
-import static org.apache.aurora.scheduler.base.TaskTestUtil.REVOCABLE_TIER;
-import static org.apache.mesos.Protos.Value.Type.RANGES;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-public class ResourcesTest {
-  @Test
-  public void testPortRangeExact() {
-    Resource portsResource = createPortRange(Pair.of(1, 5));
-    Set<Integer> ports = Resources.from(createOffer(portsResource)).getPorts(5);
-    assertEquals(5, ports.size());
-  }
-
-  @Test
-  public void testOnePortAvailable() {
-    Resource portsResource = createPortRange(Pair.of(3, 3));
-    Set<Integer> ports = Resources.from(createOffer(portsResource)).getPorts(1);
-    assertEquals(1, ports.size());
-  }
-
-  @Test
-  public void testPortRangeAbundance() {
-    Resource portsResource = createPortRange(Pair.of(1, 10));
-    Set<Integer> ports = Resources.from(createOffer(portsResource)).getPorts(5);
-    assertEquals(5, ports.size());
-  }
-
-  @Test
-  public void testPortRangeExhaust() {
-    Resource portsResource = createPortRanges(Pair.of(1, 2), Pair.of(10, 15));
-
-    Set<Integer> ports = Resources.from(createOffer(portsResource)).getPorts(7);
-    assertEquals(7, ports.size());
-
-    ports = Resources.from(createOffer(portsResource)).getPorts(8);
-    assertEquals(8, ports.size());
-
-    try {
-      Resources.from(createOffer(portsResource)).getPorts(9);
-      fail("Ports should not have been sufficient");
-    } catch (InsufficientResourcesException e) {
-      // Expected.
-    }
-  }
-
-  @Test
-  public void testGetNoPorts() {
-    Resource portsResource = createPortRange(Pair.of(1, 5));
-    assertEquals(ImmutableSet.of(), Resources.from(createOffer(portsResource)).getPorts(0));
-  }
-
-  @Test(expected = Resources.InsufficientResourcesException.class)
-  public void testPortRangeScarcity() {
-    Resource portsResource = createPortRange(Pair.of(1, 2));
-    Resources.from(createOffer(portsResource)).getPorts(5);
-  }
-
-  @Test
-  public void testGetSlot() {
-    ImmutableList<Resource> resources = ImmutableList.<Resource>builder()
-        .add(makeMesosResource(CPUS, 8.0, false))
-        .add(makeMesosResource(RAM_MB, 1024, false))
-        .add(makeMesosResource(DISK_MB, 2048, false))
-        .add(createPortRange(Pair.of(1, 10)))
-        .build();
-
-    ResourceSlot expected = new ResourceSlot(8.0, Amount.of(1024L, MB), Amount.of(2048L, MB), 10);
-    assertEquals(expected, Resources.from(createOffer(resources)).slot());
-  }
-
-  @Test
-  public void testMissingResourcesHandledGracefully() {
-    ImmutableList<Resource> resources = ImmutableList.<Resource>builder().build();
-    assertEquals(ResourceSlot.NONE, Resources.from(createOffer(resources)).slot());
-  }
-
-  @Test
-  public void testFilter() {
-    ImmutableList<Resource> resources = ImmutableList.<Resource>builder()
-        .add(makeMesosResource(CPUS, 8.0, true))
-        .add(makeMesosResource(RAM_MB, 1024, false))
-        .build();
-
-    assertEquals(
-        new ResourceSlot(8.0, Amount.of(1024L, MB), Amount.of(0L, MB), 0),
-        Resources.from(createOffer(resources)).filter(Resources.REVOCABLE).slot());
-  }
-
-  @Test
-  public void testFilterByTier() {
-    ImmutableList<Resource> resources = ImmutableList.<Resource>builder()
-        .add(makeMesosResource(CPUS, 8.0, true))
-        .add(makeMesosResource(CPUS, 8.0, false))
-        .add(makeMesosResource(RAM_MB, 1024, false))
-        .build();
-
-    assertEquals(
-        new ResourceSlot(8.0, Amount.of(1024L, MB), Amount.of(0L, MB), 0),
-        Resources.from(createOffer(resources)).filter(REVOCABLE_TIER).slot());
-
-    assertEquals(
-        new ResourceSlot(8.0, Amount.of(1024L, MB), Amount.of(0L, MB), 0),
-        Resources.from(createOffer(resources)).filter(DEV_TIER).slot());
-  }
-
-  private Resource createPortRange(Pair<Integer, Integer> range) {
-    return createPortRanges(ImmutableSet.of(range));
-  }
-
-  private Resource createPortRanges(Pair<Integer, Integer> rangeA, Pair<Integer, Integer> rangeB) {
-    return createPortRanges(
-        ImmutableSet.<Pair<Integer, Integer>>builder().add(rangeA).add(rangeB).build());
-  }
-
-  private Resource createPortRanges(Set<Pair<Integer, Integer>> ports) {
-    Ranges.Builder ranges = Ranges.newBuilder();
-    for (Pair<Integer, Integer> range : ports) {
-      ranges.addRange(Range.newBuilder().setBegin(range.getFirst()).setEnd(range.getSecond()));
-    }
-
-    return Resource.newBuilder()
-        .setName(PORTS.getName())
-        .setType(RANGES)
-        .setRanges(ranges)
-        .build();
-  }
-
-  private static Protos.Offer createOffer(Resource resource) {
-    return createOffer(ImmutableList.of(resource));
-  }
-
-  private static Protos.Offer createOffer(Iterable<Resource> resources) {
-    return Protos.Offer.newBuilder()
-        .setId(Protos.OfferID.newBuilder().setValue("offer-id"))
-        .setFrameworkId(Protos.FrameworkID.newBuilder().setValue("framework-id"))
-        .setSlaveId(Protos.SlaveID.newBuilder().setValue("slave-id"))
-        .setHostname("hostname")
-        .addAllResources(resources).build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/test/java/org/apache/aurora/scheduler/app/SchedulerIT.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/app/SchedulerIT.java b/src/test/java/org/apache/aurora/scheduler/app/SchedulerIT.java
index b1426bc..89b2813 100644
--- a/src/test/java/org/apache/aurora/scheduler/app/SchedulerIT.java
+++ b/src/test/java/org/apache/aurora/scheduler/app/SchedulerIT.java
@@ -59,7 +59,6 @@ import org.apache.aurora.gen.storage.Snapshot;
 import org.apache.aurora.gen.storage.Transaction;
 import org.apache.aurora.gen.storage.storageConstants;
 import org.apache.aurora.scheduler.AppStartup;
-import org.apache.aurora.scheduler.ResourceSlot;
 import org.apache.aurora.scheduler.TierModule;
 import org.apache.aurora.scheduler.base.TaskTestUtil;
 import org.apache.aurora.scheduler.configuration.executor.ExecutorSettings;
@@ -71,6 +70,7 @@ import org.apache.aurora.scheduler.log.Log.Stream;
 import org.apache.aurora.scheduler.mesos.DriverFactory;
 import org.apache.aurora.scheduler.mesos.DriverSettings;
 import org.apache.aurora.scheduler.mesos.TestExecutorSettings;
+import org.apache.aurora.scheduler.resources.ResourceSlot;
 import org.apache.aurora.scheduler.storage.backup.BackupModule;
 import org.apache.aurora.scheduler.storage.entities.IScheduledTask;
 import org.apache.aurora.scheduler.storage.entities.IServerInfo;

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/test/java/org/apache/aurora/scheduler/app/local/simulator/ClusterSimulatorModule.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/app/local/simulator/ClusterSimulatorModule.java b/src/test/java/org/apache/aurora/scheduler/app/local/simulator/ClusterSimulatorModule.java
index ea87688..48a665f 100644
--- a/src/test/java/org/apache/aurora/scheduler/app/local/simulator/ClusterSimulatorModule.java
+++ b/src/test/java/org/apache/aurora/scheduler/app/local/simulator/ClusterSimulatorModule.java
@@ -29,11 +29,11 @@ import org.apache.mesos.Protos.Offer;
 
 import static java.util.Objects.requireNonNull;
 
-import static org.apache.aurora.scheduler.ResourceType.CPUS;
-import static org.apache.aurora.scheduler.ResourceType.DISK_MB;
-import static org.apache.aurora.scheduler.ResourceType.PORTS;
-import static org.apache.aurora.scheduler.ResourceType.RAM_MB;
 import static org.apache.aurora.scheduler.configuration.ConfigurationManager.DEDICATED_ATTRIBUTE;
+import static org.apache.aurora.scheduler.resources.ResourceType.CPUS;
+import static org.apache.aurora.scheduler.resources.ResourceType.DISK_MB;
+import static org.apache.aurora.scheduler.resources.ResourceType.PORTS;
+import static org.apache.aurora.scheduler.resources.ResourceType.RAM_MB;
 import static org.apache.mesos.Protos.Value.Type.RANGES;
 import static org.apache.mesos.Protos.Value.Type.SCALAR;
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/test/java/org/apache/aurora/scheduler/events/NotifyingSchedulingFilterTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/events/NotifyingSchedulingFilterTest.java b/src/test/java/org/apache/aurora/scheduler/events/NotifyingSchedulingFilterTest.java
index b9f9f52..1474fa9 100644
--- a/src/test/java/org/apache/aurora/scheduler/events/NotifyingSchedulingFilterTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/events/NotifyingSchedulingFilterTest.java
@@ -21,7 +21,6 @@ import org.apache.aurora.common.testing.easymock.EasyMockTest;
 import org.apache.aurora.gen.HostAttributes;
 import org.apache.aurora.gen.MaintenanceMode;
 import org.apache.aurora.gen.TaskConfig;
-import org.apache.aurora.scheduler.ResourceSlot;
 import org.apache.aurora.scheduler.base.TaskGroupKey;
 import org.apache.aurora.scheduler.events.PubsubEvent.Vetoed;
 import org.apache.aurora.scheduler.filter.AttributeAggregate;
@@ -29,6 +28,7 @@ import org.apache.aurora.scheduler.filter.SchedulingFilter;
 import org.apache.aurora.scheduler.filter.SchedulingFilter.ResourceRequest;
 import org.apache.aurora.scheduler.filter.SchedulingFilter.UnusedResource;
 import org.apache.aurora.scheduler.filter.SchedulingFilter.Veto;
+import org.apache.aurora.scheduler.resources.ResourceSlot;
 import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 import org.apache.aurora.scheduler.storage.entities.ITaskConfig;
 import org.junit.Before;

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/test/java/org/apache/aurora/scheduler/filter/SchedulingFilterImplTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/filter/SchedulingFilterImplTest.java b/src/test/java/org/apache/aurora/scheduler/filter/SchedulingFilterImplTest.java
index 17101f1..1d9d5fc 100644
--- a/src/test/java/org/apache/aurora/scheduler/filter/SchedulingFilterImplTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/filter/SchedulingFilterImplTest.java
@@ -32,8 +32,6 @@ import org.apache.aurora.gen.MaintenanceMode;
 import org.apache.aurora.gen.TaskConfig;
 import org.apache.aurora.gen.TaskConstraint;
 import org.apache.aurora.gen.ValueConstraint;
-import org.apache.aurora.scheduler.ResourceSlot;
-import org.apache.aurora.scheduler.Resources;
 import org.apache.aurora.scheduler.base.JobKeys;
 import org.apache.aurora.scheduler.filter.SchedulingFilter.ResourceRequest;
 import org.apache.aurora.scheduler.filter.SchedulingFilter.UnusedResource;
@@ -42,6 +40,8 @@ import org.apache.aurora.scheduler.filter.SchedulingFilter.VetoGroup;
 import org.apache.aurora.scheduler.filter.SchedulingFilter.VetoType;
 import org.apache.aurora.scheduler.mesos.Offers;
 import org.apache.aurora.scheduler.mesos.TaskExecutors;
+import org.apache.aurora.scheduler.resources.ResourceSlot;
+import org.apache.aurora.scheduler.resources.Resources;
 import org.apache.aurora.scheduler.storage.entities.IAttribute;
 import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 import org.apache.aurora.scheduler.storage.entities.IJobKey;

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/test/java/org/apache/aurora/scheduler/mesos/MesosTaskFactoryImplTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/mesos/MesosTaskFactoryImplTest.java b/src/test/java/org/apache/aurora/scheduler/mesos/MesosTaskFactoryImplTest.java
index 4f5ac15..bf18d5d 100644
--- a/src/test/java/org/apache/aurora/scheduler/mesos/MesosTaskFactoryImplTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/mesos/MesosTaskFactoryImplTest.java
@@ -29,14 +29,14 @@ import org.apache.aurora.gen.DockerParameter;
 import org.apache.aurora.gen.MesosContainer;
 import org.apache.aurora.gen.ServerInfo;
 import org.apache.aurora.gen.TaskConfig;
-import org.apache.aurora.scheduler.ResourceSlot;
-import org.apache.aurora.scheduler.ResourceType;
-import org.apache.aurora.scheduler.Resources;
 import org.apache.aurora.scheduler.TierManager;
 import org.apache.aurora.scheduler.base.TaskTestUtil;
 import org.apache.aurora.scheduler.configuration.executor.ExecutorConfig;
 import org.apache.aurora.scheduler.configuration.executor.ExecutorSettings;
 import org.apache.aurora.scheduler.mesos.MesosTaskFactory.MesosTaskFactoryImpl;
+import org.apache.aurora.scheduler.resources.ResourceSlot;
+import org.apache.aurora.scheduler.resources.ResourceType;
+import org.apache.aurora.scheduler.resources.Resources;
 import org.apache.aurora.scheduler.storage.entities.IAssignedTask;
 import org.apache.aurora.scheduler.storage.entities.IJobKey;
 import org.apache.aurora.scheduler.storage.entities.IServerInfo;
@@ -56,7 +56,6 @@ import org.apache.mesos.Protos.Volume.Mode;
 import org.junit.Before;
 import org.junit.Test;
 
-import static org.apache.aurora.scheduler.ResourceSlot.makeMesosRangeResource;
 import static org.apache.aurora.scheduler.base.TaskTestUtil.DEV_TIER;
 import static org.apache.aurora.scheduler.base.TaskTestUtil.REVOCABLE_TIER;
 import static org.apache.aurora.scheduler.mesos.MesosTaskFactory.MesosTaskFactoryImpl.DEFAULT_PORT_PROTOCOL;
@@ -66,6 +65,7 @@ import static org.apache.aurora.scheduler.mesos.TaskExecutors.NO_OVERHEAD_EXECUT
 import static org.apache.aurora.scheduler.mesos.TaskExecutors.SOME_OVERHEAD_EXECUTOR;
 import static org.apache.aurora.scheduler.mesos.TestExecutorSettings.THERMOS_CONFIG;
 import static org.apache.aurora.scheduler.mesos.TestExecutorSettings.THERMOS_EXECUTOR;
+import static org.apache.aurora.scheduler.resources.ResourceSlot.makeMesosRangeResource;
 import static org.easymock.EasyMock.expect;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/test/java/org/apache/aurora/scheduler/mesos/Offers.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/mesos/Offers.java b/src/test/java/org/apache/aurora/scheduler/mesos/Offers.java
index 13f41e3..a7fb4b0 100644
--- a/src/test/java/org/apache/aurora/scheduler/mesos/Offers.java
+++ b/src/test/java/org/apache/aurora/scheduler/mesos/Offers.java
@@ -24,10 +24,10 @@ import org.apache.mesos.Protos.Value.Ranges;
 import org.apache.mesos.Protos.Value.Scalar;
 import org.apache.mesos.Protos.Value.Type;
 
-import static org.apache.aurora.scheduler.ResourceType.CPUS;
-import static org.apache.aurora.scheduler.ResourceType.DISK_MB;
-import static org.apache.aurora.scheduler.ResourceType.PORTS;
-import static org.apache.aurora.scheduler.ResourceType.RAM_MB;
+import static org.apache.aurora.scheduler.resources.ResourceType.CPUS;
+import static org.apache.aurora.scheduler.resources.ResourceType.DISK_MB;
+import static org.apache.aurora.scheduler.resources.ResourceType.PORTS;
+import static org.apache.aurora.scheduler.resources.ResourceType.RAM_MB;
 
 public final class Offers {
   private Offers() {

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/test/java/org/apache/aurora/scheduler/mesos/TaskExecutors.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/mesos/TaskExecutors.java b/src/test/java/org/apache/aurora/scheduler/mesos/TaskExecutors.java
index 155ece4..651cfa7 100644
--- a/src/test/java/org/apache/aurora/scheduler/mesos/TaskExecutors.java
+++ b/src/test/java/org/apache/aurora/scheduler/mesos/TaskExecutors.java
@@ -15,8 +15,8 @@ package org.apache.aurora.scheduler.mesos;
 
 import org.apache.aurora.common.quantity.Amount;
 import org.apache.aurora.common.quantity.Data;
-import org.apache.aurora.scheduler.ResourceSlot;
 import org.apache.aurora.scheduler.configuration.executor.ExecutorSettings;
+import org.apache.aurora.scheduler.resources.ResourceSlot;
 
 /**
  * Utility class to contain constants related to setting up executor settings.

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/test/java/org/apache/aurora/scheduler/preemptor/PreemptionVictimFilterTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/preemptor/PreemptionVictimFilterTest.java b/src/test/java/org/apache/aurora/scheduler/preemptor/PreemptionVictimFilterTest.java
index bb1b01e..0739877 100644
--- a/src/test/java/org/apache/aurora/scheduler/preemptor/PreemptionVictimFilterTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/preemptor/PreemptionVictimFilterTest.java
@@ -34,13 +34,13 @@ import org.apache.aurora.gen.ScheduledTask;
 import org.apache.aurora.gen.TaskConfig;
 import org.apache.aurora.gen.TaskEvent;
 import org.apache.aurora.scheduler.HostOffer;
-import org.apache.aurora.scheduler.ResourceSlot;
 import org.apache.aurora.scheduler.TierInfo;
 import org.apache.aurora.scheduler.TierManager;
 import org.apache.aurora.scheduler.filter.SchedulingFilter;
 import org.apache.aurora.scheduler.filter.SchedulingFilter.Veto;
 import org.apache.aurora.scheduler.filter.SchedulingFilterImpl;
 import org.apache.aurora.scheduler.mesos.TaskExecutors;
+import org.apache.aurora.scheduler.resources.ResourceSlot;
 import org.apache.aurora.scheduler.stats.CachedCounters;
 import org.apache.aurora.scheduler.storage.entities.IAssignedTask;
 import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
@@ -55,12 +55,12 @@ import org.junit.Test;
 
 import static org.apache.aurora.gen.MaintenanceMode.NONE;
 import static org.apache.aurora.gen.ScheduleStatus.RUNNING;
-import static org.apache.aurora.scheduler.ResourceType.CPUS;
 import static org.apache.aurora.scheduler.base.TaskTestUtil.DEV_TIER;
 import static org.apache.aurora.scheduler.base.TaskTestUtil.PREFERRED_TIER;
 import static org.apache.aurora.scheduler.base.TaskTestUtil.REVOCABLE_TIER;
 import static org.apache.aurora.scheduler.filter.AttributeAggregate.EMPTY;
 import static org.apache.aurora.scheduler.preemptor.PreemptorMetrics.MISSING_ATTRIBUTES_NAME;
+import static org.apache.aurora.scheduler.resources.ResourceType.CPUS;
 import static org.apache.mesos.Protos.Offer;
 import static org.apache.mesos.Protos.Resource;
 import static org.easymock.EasyMock.expect;

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/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 aedeab3..fc01821 100644
--- a/src/test/java/org/apache/aurora/scheduler/quota/QuotaManagerImplTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/quota/QuotaManagerImplTest.java
@@ -54,10 +54,10 @@ 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;
+import static org.apache.aurora.scheduler.resources.ResourceAggregates.EMPTY;
 import static org.easymock.EasyMock.anyObject;
 import static org.easymock.EasyMock.expect;
 import static org.junit.Assert.assertEquals;

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/test/java/org/apache/aurora/scheduler/resources/AcceptedOfferTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/resources/AcceptedOfferTest.java b/src/test/java/org/apache/aurora/scheduler/resources/AcceptedOfferTest.java
new file mode 100644
index 0000000..319ffea
--- /dev/null
+++ b/src/test/java/org/apache/aurora/scheduler/resources/AcceptedOfferTest.java
@@ -0,0 +1,305 @@
+/**
+ * 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.resources;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+
+import org.apache.aurora.common.quantity.Amount;
+import org.apache.aurora.common.quantity.Data;
+import org.apache.aurora.scheduler.TierInfo;
+import org.apache.aurora.scheduler.base.TaskTestUtil;
+import org.apache.mesos.Protos;
+import org.apache.mesos.Protos.Resource;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class AcceptedOfferTest {
+
+  private static final Optional<String> TEST_ROLE = Optional.of("test-role");
+  private static final Optional<String> ABSENT_ROLE = Optional.absent();
+  private static final ResourceSlot TASK_SLOT = new ResourceSlot(
+      4, Amount.of(100L, Data.MB), Amount.of(200L, Data.MB), 0);
+  private static final ResourceSlot EXECUTOR_SLOT = new ResourceSlot(
+      0.25, Amount.of(25L, Data.MB), Amount.of(75L, Data.MB), 0);
+  private static final ResourceSlot TOTAL_SLOT = EXECUTOR_SLOT.add(TASK_SLOT);
+  private static final Integer[] TASK_PORTS = {80, 90};
+  private static final Set<Integer> TASK_PORTS_SET = ImmutableSet.copyOf(TASK_PORTS);
+
+  @Test
+  public void testReservedPredicates() {
+    Protos.Resource withRole = makeScalar(ResourceType.CPUS.getName(), TEST_ROLE, false, 1.0);
+    assertTrue(AcceptedOffer.RESERVED.apply(withRole));
+    assertFalse(AcceptedOffer.NOT_RESERVED.apply(withRole));
+    Protos.Resource absentRole = makeScalar(ResourceType.CPUS.getName(), ABSENT_ROLE, false, 1.0);
+    assertFalse(AcceptedOffer.RESERVED.apply(absentRole));
+    assertTrue(AcceptedOffer.NOT_RESERVED.apply(absentRole));
+  }
+
+  @Test
+  public void testAllocateEmpty() {
+    AcceptedOffer acceptedOffer = AcceptedOffer.create(
+        fakeOffer(Collections.emptyList()),
+        ResourceSlot.NONE,
+        ResourceSlot.NONE,
+        ImmutableSet.of(),
+        TaskTestUtil.DEV_TIER);
+    assertEquals(Collections.emptyList(), acceptedOffer.getTaskResources());
+    assertEquals(Collections.emptyList(), acceptedOffer.getExecutorResources());
+  }
+
+  @Test
+  public void testAllocateRange() {
+    List<Resource> resources = ImmutableList.<Resource>builder()
+        .add(makePortResource(Optional.absent(), 80, 81, 90, 91, 92, 93))
+        .add(makePortResource(TEST_ROLE, 100, 101))
+        .build();
+    AcceptedOffer acceptedOffer = AcceptedOffer.create(
+        fakeOffer(resources),
+        ResourceSlot.NONE,
+        ResourceSlot.NONE,
+        ImmutableSet.of(80, 90, 100),
+        TaskTestUtil.DEV_TIER);
+
+    List<Resource> expected = ImmutableList.<Resource>builder()
+        // Because we prefer reserved resources and handle them before non-reserved resources,
+        // result should have ports for the reserved resources first.
+        .add(makePortResource(TEST_ROLE, 100))
+        .add(makePortResource(Optional.absent(), 80, 90))
+        .build();
+    assertEquals(expected, acceptedOffer.getTaskResources());
+    assertEquals(Collections.emptyList(), acceptedOffer.getExecutorResources());
+  }
+
+  @Test(expected = Resources.InsufficientResourcesException.class)
+  public void testAllocateRangeInsufficent() {
+    List<Resource> resources = ImmutableList.of(
+        makePortResource(ABSENT_ROLE, 80),
+        makePortResource(ABSENT_ROLE, 100, 101));
+    AcceptedOffer.create(
+        fakeOffer(resources),
+        ResourceSlot.NONE,
+        ResourceSlot.NONE,
+        ImmutableSet.of(80, 90, 100),
+        TaskTestUtil.DEV_TIER);
+  }
+
+  @Test
+  public void testAllocateSingleRole() {
+    runAllocateSingleRole(ABSENT_ROLE, false);
+    runAllocateSingleRole(ABSENT_ROLE, true);
+    runAllocateSingleRole(TEST_ROLE, false);
+    runAllocateSingleRole(TEST_ROLE, true);
+  }
+
+  private void runAllocateSingleRole(Optional<String> role, boolean cpuRevocable) {
+    List<Resource> resources = ImmutableList.<Resource>builder()
+        .add(makeScalar(
+            ResourceType.CPUS.getName(), role, cpuRevocable, TOTAL_SLOT.getNumCpus()))
+        .add(makeScalar(
+            ResourceType.RAM_MB.getName(), role, false, TOTAL_SLOT.getRam().as(Data.MB)))
+        .add(makeScalar(
+            ResourceType.DISK_MB.getName(), role, false, TOTAL_SLOT.getDisk().as(Data.MB)))
+        .add(makePortResource(role, TASK_PORTS))
+        .build();
+    Protos.Offer offer = fakeOffer(resources);
+
+    AcceptedOffer offerAllocation = AcceptedOffer.create(
+        offer, TASK_SLOT, EXECUTOR_SLOT, TASK_PORTS_SET, new TierInfo(false, cpuRevocable));
+
+    List<Resource> taskList = ImmutableList.<Resource>builder()
+        .add(makeScalar(ResourceType.CPUS.getName(), role, cpuRevocable, TASK_SLOT.getNumCpus()))
+        .add(makeScalar(ResourceType.RAM_MB.getName(), role, false, TASK_SLOT.getRam().as(Data.MB)))
+        .add(makeScalar(
+            ResourceType.DISK_MB.getName(), role, false, TASK_SLOT.getDisk().as(Data.MB)))
+        .add(makePortResource(role, TASK_PORTS))
+        .build();
+    assertEquals(taskList, offerAllocation.getTaskResources());
+
+    List<Resource> executorList = ImmutableList.<Resource>builder()
+        .add(makeScalar(
+            ResourceType.CPUS.getName(), role, cpuRevocable, EXECUTOR_SLOT.getNumCpus()))
+        .add(makeScalar(
+            ResourceType.RAM_MB.getName(), role, false, EXECUTOR_SLOT.getRam().as(Data.MB)))
+        .add(makeScalar(
+            ResourceType.DISK_MB.getName(), role, false, EXECUTOR_SLOT.getDisk().as(Data.MB)))
+        .build();
+    assertEquals(executorList, offerAllocation.getExecutorResources());
+  }
+
+  @Test(expected = Resources.InsufficientResourcesException.class)
+  public void testAllocateSingleRoleInsufficient() {
+    List<Resource> resources = ImmutableList.<Resource>builder()
+        // EXECUTOR_SLOT's CPU is not included here.
+        .add(makeScalar(ResourceType.CPUS.getName(), TEST_ROLE, false, TASK_SLOT.getNumCpus()))
+        .add(makeScalar(
+            ResourceType.RAM_MB.getName(), TEST_ROLE, false, TOTAL_SLOT.getRam().as(Data.MB)))
+        .add(makeScalar(
+            ResourceType.DISK_MB.getName(), TEST_ROLE, false, TOTAL_SLOT.getDisk().as(Data.MB)))
+        .add(makePortResource(TEST_ROLE, TASK_PORTS))
+        .build();
+    Protos.Offer offer = fakeOffer(resources);
+
+    AcceptedOffer.create(
+        offer, TASK_SLOT, EXECUTOR_SLOT, TASK_PORTS_SET, new TierInfo(false, false));
+  }
+
+  @Test
+  public void testMultipleRoles() {
+    runMultipleRoles(false);
+    runMultipleRoles(true);
+  }
+
+  private void runMultipleRoles(boolean cpuRevocable) {
+    List<Resource> resources = ImmutableList.<Resource>builder()
+        // Make cpus come from two roles.
+        .add(makeScalar(
+            ResourceType.CPUS.getName(),
+            TEST_ROLE,
+            cpuRevocable,
+            EXECUTOR_SLOT.getNumCpus()))
+        .add(makeScalar(
+            ResourceType.CPUS.getName(),
+            ABSENT_ROLE,
+            cpuRevocable,
+            TASK_SLOT.getNumCpus()))
+        // Make ram come from default role
+        .add(makeScalar(
+            ResourceType.RAM_MB.getName(),
+            ABSENT_ROLE,
+            false,
+            TOTAL_SLOT.getRam().as(Data.MB)))
+        // Make disk come from non-default role.
+        .add(makeScalar(
+            ResourceType.DISK_MB.getName(),
+            TEST_ROLE,
+            false,
+            TOTAL_SLOT.getDisk().as(Data.MB)))
+        .add(makePortResource(TEST_ROLE, 80))
+        .add(makePortResource(ABSENT_ROLE, 90))
+        .build();
+
+    Protos.Offer offer = fakeOffer(resources);
+
+    AcceptedOffer offerAllocation = AcceptedOffer.create(
+        offer, TASK_SLOT, EXECUTOR_SLOT, TASK_PORTS_SET, new TierInfo(false, cpuRevocable));
+
+    List<Resource> taskList = ImmutableList.<Resource>builder()
+        // We intentionally sliced the offer resource to not align with TASK_SLOT's num cpus.
+        .add(makeScalar(
+            ResourceType.CPUS.getName(), TEST_ROLE, cpuRevocable, EXECUTOR_SLOT.getNumCpus()))
+        .add(makeScalar(
+            ResourceType.CPUS.getName(),
+            ABSENT_ROLE,
+            cpuRevocable,
+            TASK_SLOT.subtract(EXECUTOR_SLOT).getNumCpus()))
+        .add(makeScalar(
+            ResourceType.RAM_MB.getName(), ABSENT_ROLE, false, TASK_SLOT.getRam().as(Data.MB)))
+        .add(makeScalar(
+            ResourceType.DISK_MB.getName(), TEST_ROLE, false, TASK_SLOT.getDisk().as(Data.MB)))
+        .add(makePortResource(TEST_ROLE, 80))
+        .add(makePortResource(ABSENT_ROLE, 90))
+        .build();
+    assertEquals(taskList, offerAllocation.getTaskResources());
+
+    List<Resource> executorList = ImmutableList.<Resource>builder()
+        .add(makeScalar(
+            ResourceType.CPUS.getName(), ABSENT_ROLE, cpuRevocable, EXECUTOR_SLOT.getNumCpus()))
+        .add(makeScalar(
+            ResourceType.RAM_MB.getName(), ABSENT_ROLE, false, EXECUTOR_SLOT.getRam().as(Data.MB)))
+        .add(makeScalar(
+            ResourceType.DISK_MB.getName(), TEST_ROLE, false, EXECUTOR_SLOT.getDisk().as(Data.MB)))
+        .build();
+    assertEquals(executorList, offerAllocation.getExecutorResources());
+  }
+
+  @Test(expected = Resources.InsufficientResourcesException.class)
+  public void testMultipleRolesInsufficient() {
+    // Similar to testMultipleRoles, but make some of cpus as revocable
+    List<Resource> resources = ImmutableList.<Resource>builder()
+        // Make cpus come from two roles.
+        .add(makeScalar(
+            ResourceType.CPUS.getName(),
+            TEST_ROLE,
+            true,
+            EXECUTOR_SLOT.getNumCpus()))
+        .add(makeScalar(
+            ResourceType.CPUS.getName(),
+            ABSENT_ROLE,
+            false,
+            TASK_SLOT.getNumCpus()))
+        // Make ram come from default role
+        .add(makeScalar(
+            ResourceType.RAM_MB.getName(),
+            ABSENT_ROLE,
+            false,
+            TOTAL_SLOT.getRam().as(Data.MB)))
+        // Make disk come from non-default role.
+        .add(makeScalar(
+            ResourceType.DISK_MB.getName(),
+            TEST_ROLE,
+            false,
+            TOTAL_SLOT.getDisk().as(Data.MB)))
+        .add(makePortResource(TEST_ROLE, 80))
+        .add(makePortResource(ABSENT_ROLE, 90))
+        .build();
+    Protos.Offer offer = fakeOffer(resources);
+    // We don't have enough resource to satisfy a non-revocable request.
+    AcceptedOffer.create(
+        offer, TASK_SLOT, EXECUTOR_SLOT, TASK_PORTS_SET, new TierInfo(false, false));
+  }
+
+  private static Resource makePortResource(Optional<String> role, Integer... values) {
+    Resource.Builder prototype = Resource.newBuilder()
+        .setType(Protos.Value.Type.RANGES)
+        .setName(ResourceType.PORTS.getName());
+    if (role.isPresent()) {
+      prototype.setRole(role.get());
+    }
+    return AcceptedOffer.makeMesosRangeResource(prototype.build(), ImmutableSet.copyOf(values));
+  }
+
+  private static Resource makeScalar(
+      String name, Optional<String> role, boolean revocable, double value) {
+    Resource.Builder resource = Resource.newBuilder()
+        .setName(name)
+        .setType(Protos.Value.Type.SCALAR)
+        .setScalar(Protos.Value.Scalar.newBuilder().setValue(value));
+    if (role.isPresent()) {
+      resource.setRole(role.get());
+    }
+    if (revocable) {
+      resource.setRevocable(Resource.RevocableInfo.getDefaultInstance());
+    }
+    return resource.build();
+  }
+
+  private static Protos.Offer fakeOffer(List<Resource> resources) {
+    return Protos.Offer.newBuilder()
+        .setId(Protos.OfferID.newBuilder().setValue("offer-id"))
+        .setFrameworkId(Protos.FrameworkID.newBuilder().setValue("framework-id"))
+        .setSlaveId(Protos.SlaveID.newBuilder().setValue("slave-id"))
+        .setHostname("hostname")
+        .addAllResources(resources)
+        .build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/test/java/org/apache/aurora/scheduler/resources/ResourceSlotTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/resources/ResourceSlotTest.java b/src/test/java/org/apache/aurora/scheduler/resources/ResourceSlotTest.java
new file mode 100644
index 0000000..8d5a143
--- /dev/null
+++ b/src/test/java/org/apache/aurora/scheduler/resources/ResourceSlotTest.java
@@ -0,0 +1,170 @@
+/**
+ * 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.resources;
+
+import java.util.Set;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+
+import org.apache.aurora.common.collections.Pair;
+import org.apache.aurora.common.quantity.Amount;
+import org.apache.aurora.common.quantity.Data;
+import org.apache.aurora.gen.TaskConfig;
+import org.apache.aurora.scheduler.storage.entities.ITaskConfig;
+import org.apache.mesos.Protos;
+import org.junit.Test;
+
+import static org.apache.aurora.scheduler.base.TaskTestUtil.DEV_TIER;
+import static org.apache.aurora.scheduler.base.TaskTestUtil.REVOCABLE_TIER;
+import static org.apache.aurora.scheduler.resources.ResourceSlot.makeMesosRangeResource;
+import static org.apache.aurora.scheduler.resources.ResourceSlot.makeMesosResource;
+import static org.apache.aurora.scheduler.resources.ResourceSlot.sum;
+import static org.apache.aurora.scheduler.resources.ResourceType.CPUS;
+import static org.apache.aurora.scheduler.resources.ResourceType.DISK_MB;
+import static org.apache.aurora.scheduler.resources.ResourceType.PORTS;
+import static org.apache.aurora.scheduler.resources.ResourceType.RAM_MB;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+
+public class ResourceSlotTest {
+
+  private static final ResourceSlot NEGATIVE_ONE =
+      new ResourceSlot(-1.0, Amount.of(-1L, Data.MB), Amount.of(-1L, Data.MB), -1);
+  private static final ResourceSlot ONE =
+      new ResourceSlot(1.0, Amount.of(1L, Data.MB), Amount.of(1L, Data.MB), 1);
+  private static final ResourceSlot TWO =
+      new ResourceSlot(2.0, Amount.of(2L, Data.MB), Amount.of(2L, Data.MB), 2);
+  private static final ResourceSlot THREE =
+      new ResourceSlot(3.0, Amount.of(3L, Data.MB), Amount.of(3L, Data.MB), 3);
+  private static final ITaskConfig TASK = ITaskConfig.build(new TaskConfig()
+      .setNumCpus(1.0)
+      .setRamMb(1024)
+      .setDiskMb(2048)
+      .setRequestedPorts(ImmutableSet.of("http", "debug")));
+
+  @Test
+  public void testMaxElements() {
+    ResourceSlot highRAM = new ResourceSlot(1, Amount.of(8L, Data.GB), Amount.of(10L, Data.MB), 0);
+    ResourceSlot rest = new ResourceSlot(10, Amount.of(1L, Data.MB), Amount.of(10L, Data.GB), 1);
+
+    ResourceSlot result = ResourceSlot.maxElements(highRAM, rest);
+    assertEquals(result.getNumCpus(), 10, 0.001);
+    assertEquals(result.getRam(), Amount.of(8L, Data.GB));
+    assertEquals(result.getDisk(), Amount.of(10L, Data.GB));
+    assertEquals(result.getNumPorts(), 1);
+  }
+
+  @Test
+  public void testSubtract() {
+    assertEquals(ONE, TWO.subtract(ONE));
+    assertEquals(TWO, THREE.subtract(ONE));
+    assertEquals(NEGATIVE_ONE, ONE.subtract(TWO));
+    assertEquals(NEGATIVE_ONE, TWO.subtract(THREE));
+  }
+
+  @Test
+  public void testAdd() {
+    assertEquals(TWO, ONE.add(ONE));
+    assertEquals(THREE, ONE.add(TWO));
+    assertEquals(THREE, TWO.add(ONE));
+  }
+
+  @Test
+  public void testSum() {
+    assertEquals(THREE, sum(ImmutableList.of(ONE, ONE, ONE)));
+  }
+
+  @Test
+  public void testToResourceListNoRevoca() {
+    ResourceSlot resources = ResourceSlot.from(TASK);
+    assertEquals(
+        ImmutableSet.of(
+            makeMesosResource(CPUS, TASK.getNumCpus(), false),
+            makeMesosResource(RAM_MB, TASK.getRamMb(), false),
+            makeMesosResource(DISK_MB, TASK.getDiskMb(), false)),
+        ImmutableSet.copyOf(resources.toResourceList(DEV_TIER)));
+  }
+
+  @Test
+  public void testToResourceListRevocable() {
+    ResourceSlot resources = ResourceSlot.from(TASK);
+    assertEquals(
+        ImmutableSet.of(
+            makeMesosResource(CPUS, TASK.getNumCpus(), true),
+            makeMesosResource(RAM_MB, TASK.getRamMb(), false),
+            makeMesosResource(DISK_MB, TASK.getDiskMb(), false)),
+        ImmutableSet.copyOf(resources.toResourceList(REVOCABLE_TIER)));
+  }
+
+  @Test
+  public void testToResourceListNoPorts() {
+    ResourceSlot resources = ResourceSlot.from(TASK);
+    assertEquals(
+        ImmutableSet.of(
+            makeMesosResource(CPUS, TASK.getNumCpus(), true),
+            makeMesosResource(RAM_MB, TASK.getRamMb(), false),
+            makeMesosResource(DISK_MB, TASK.getDiskMb(), false)),
+        ImmutableSet.copyOf(resources.toResourceList(REVOCABLE_TIER)));
+  }
+
+  @Test
+  public void testRangeResourceEmpty() {
+    expectRanges(ImmutableSet.of(), ImmutableSet.of());
+  }
+
+  @Test
+  public void testRangeResourceOneEntry() {
+    expectRanges(ImmutableSet.of(Pair.of(5L, 5L)), ImmutableSet.of(5));
+    expectRanges(ImmutableSet.of(Pair.of(0L, 0L)), ImmutableSet.of(0));
+  }
+
+  @Test
+  public void testRangeResourceNonContiguous() {
+    expectRanges(ImmutableSet.of(Pair.of(1L, 1L), Pair.of(3L, 3L), Pair.of(5L, 5L)),
+        ImmutableSet.of(5, 1, 3));
+  }
+
+  @Test
+  public void testRangeResourceContiguous() {
+    expectRanges(ImmutableSet.of(Pair.of(1L, 2L), Pair.of(4L, 5L), Pair.of(7L, 9L)),
+        ImmutableSet.of(8, 2, 4, 5, 7, 9, 1));
+  }
+
+  @Test
+  public void testEqualsBadType() {
+    ResourceSlot resources = ResourceSlot.from(TASK);
+    assertNotEquals(resources, "Hello");
+    assertNotEquals(resources, null);
+  }
+
+  @Test
+  public void testOrder() {
+    assertEquals(
+        ImmutableList.of(ONE, TWO, THREE, THREE),
+        ResourceSlot.ORDER.sortedCopy(ImmutableList.of(THREE, ONE, TWO, THREE)));
+  }
+
+  private void expectRanges(Set<Pair<Long, Long>> expected, Set<Integer> values) {
+    Protos.Resource resource = makeMesosRangeResource(PORTS, values);
+    assertEquals(Protos.Value.Type.RANGES, resource.getType());
+    assertEquals(PORTS.getName(), resource.getName());
+
+    Set<Pair<Long, Long>> actual = ImmutableSet.copyOf(Iterables.transform(
+        resource.getRanges().getRangeList(),
+        range -> Pair.of(range.getBegin(), range.getEnd())));
+    assertEquals(expected, actual);
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/test/java/org/apache/aurora/scheduler/resources/ResourcesTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/resources/ResourcesTest.java b/src/test/java/org/apache/aurora/scheduler/resources/ResourcesTest.java
new file mode 100644
index 0000000..bc2700c
--- /dev/null
+++ b/src/test/java/org/apache/aurora/scheduler/resources/ResourcesTest.java
@@ -0,0 +1,176 @@
+/**
+ * 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.resources;
+
+import java.util.Set;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+
+import org.apache.aurora.common.collections.Pair;
+import org.apache.aurora.common.quantity.Amount;
+import org.apache.aurora.scheduler.resources.Resources.InsufficientResourcesException;
+import org.apache.mesos.Protos;
+import org.apache.mesos.Protos.Resource;
+import org.apache.mesos.Protos.Value.Range;
+import org.apache.mesos.Protos.Value.Ranges;
+import org.junit.Test;
+
+import static org.apache.aurora.common.quantity.Data.MB;
+import static org.apache.aurora.scheduler.base.TaskTestUtil.DEV_TIER;
+import static org.apache.aurora.scheduler.base.TaskTestUtil.REVOCABLE_TIER;
+import static org.apache.aurora.scheduler.resources.ResourceSlot.makeMesosResource;
+import static org.apache.aurora.scheduler.resources.ResourceType.CPUS;
+import static org.apache.aurora.scheduler.resources.ResourceType.DISK_MB;
+import static org.apache.aurora.scheduler.resources.ResourceType.PORTS;
+import static org.apache.aurora.scheduler.resources.ResourceType.RAM_MB;
+import static org.apache.mesos.Protos.Value.Type.RANGES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class ResourcesTest {
+  @Test
+  public void testPortRangeExact() {
+    Resource portsResource = createPortRange(Pair.of(1, 5));
+    Set<Integer> ports = Resources.from(createOffer(portsResource)).getPorts(5);
+    assertEquals(5, ports.size());
+  }
+
+  @Test
+  public void testOnePortAvailable() {
+    Resource portsResource = createPortRange(Pair.of(3, 3));
+    Set<Integer> ports = Resources.from(createOffer(portsResource)).getPorts(1);
+    assertEquals(1, ports.size());
+  }
+
+  @Test
+  public void testPortRangeAbundance() {
+    Resource portsResource = createPortRange(Pair.of(1, 10));
+    Set<Integer> ports = Resources.from(createOffer(portsResource)).getPorts(5);
+    assertEquals(5, ports.size());
+  }
+
+  @Test
+  public void testPortRangeExhaust() {
+    Resource portsResource = createPortRanges(Pair.of(1, 2), Pair.of(10, 15));
+
+    Set<Integer> ports = Resources.from(createOffer(portsResource)).getPorts(7);
+    assertEquals(7, ports.size());
+
+    ports = Resources.from(createOffer(portsResource)).getPorts(8);
+    assertEquals(8, ports.size());
+
+    try {
+      Resources.from(createOffer(portsResource)).getPorts(9);
+      fail("Ports should not have been sufficient");
+    } catch (InsufficientResourcesException e) {
+      // Expected.
+    }
+  }
+
+  @Test
+  public void testGetNoPorts() {
+    Resource portsResource = createPortRange(Pair.of(1, 5));
+    assertEquals(ImmutableSet.of(), Resources.from(createOffer(portsResource)).getPorts(0));
+  }
+
+  @Test(expected = Resources.InsufficientResourcesException.class)
+  public void testPortRangeScarcity() {
+    Resource portsResource = createPortRange(Pair.of(1, 2));
+    Resources.from(createOffer(portsResource)).getPorts(5);
+  }
+
+  @Test
+  public void testGetSlot() {
+    ImmutableList<Resource> resources = ImmutableList.<Resource>builder()
+        .add(makeMesosResource(CPUS, 8.0, false))
+        .add(makeMesosResource(RAM_MB, 1024, false))
+        .add(makeMesosResource(DISK_MB, 2048, false))
+        .add(createPortRange(Pair.of(1, 10)))
+        .build();
+
+    ResourceSlot expected = new ResourceSlot(8.0, Amount.of(1024L, MB), Amount.of(2048L, MB), 10);
+    assertEquals(expected, Resources.from(createOffer(resources)).slot());
+  }
+
+  @Test
+  public void testMissingResourcesHandledGracefully() {
+    ImmutableList<Resource> resources = ImmutableList.<Resource>builder().build();
+    assertEquals(ResourceSlot.NONE, Resources.from(createOffer(resources)).slot());
+  }
+
+  @Test
+  public void testFilter() {
+    ImmutableList<Resource> resources = ImmutableList.<Resource>builder()
+        .add(makeMesosResource(CPUS, 8.0, true))
+        .add(makeMesosResource(RAM_MB, 1024, false))
+        .build();
+
+    assertEquals(
+        new ResourceSlot(8.0, Amount.of(1024L, MB), Amount.of(0L, MB), 0),
+        Resources.from(createOffer(resources)).filter(Resources.REVOCABLE).slot());
+  }
+
+  @Test
+  public void testFilterByTier() {
+    ImmutableList<Resource> resources = ImmutableList.<Resource>builder()
+        .add(makeMesosResource(CPUS, 8.0, true))
+        .add(makeMesosResource(CPUS, 8.0, false))
+        .add(makeMesosResource(RAM_MB, 1024, false))
+        .build();
+
+    assertEquals(
+        new ResourceSlot(8.0, Amount.of(1024L, MB), Amount.of(0L, MB), 0),
+        Resources.from(createOffer(resources)).filter(REVOCABLE_TIER).slot());
+
+    assertEquals(
+        new ResourceSlot(8.0, Amount.of(1024L, MB), Amount.of(0L, MB), 0),
+        Resources.from(createOffer(resources)).filter(DEV_TIER).slot());
+  }
+
+  private Resource createPortRange(Pair<Integer, Integer> range) {
+    return createPortRanges(ImmutableSet.of(range));
+  }
+
+  private Resource createPortRanges(Pair<Integer, Integer> rangeA, Pair<Integer, Integer> rangeB) {
+    return createPortRanges(
+        ImmutableSet.<Pair<Integer, Integer>>builder().add(rangeA).add(rangeB).build());
+  }
+
+  private Resource createPortRanges(Set<Pair<Integer, Integer>> ports) {
+    Ranges.Builder ranges = Ranges.newBuilder();
+    for (Pair<Integer, Integer> range : ports) {
+      ranges.addRange(Range.newBuilder().setBegin(range.getFirst()).setEnd(range.getSecond()));
+    }
+
+    return Resource.newBuilder()
+        .setName(PORTS.getName())
+        .setType(RANGES)
+        .setRanges(ranges)
+        .build();
+  }
+
+  private static Protos.Offer createOffer(Resource resource) {
+    return createOffer(ImmutableList.of(resource));
+  }
+
+  private static Protos.Offer createOffer(Iterable<Resource> resources) {
+    return Protos.Offer.newBuilder()
+        .setId(Protos.OfferID.newBuilder().setValue("offer-id"))
+        .setFrameworkId(Protos.FrameworkID.newBuilder().setValue("framework-id"))
+        .setSlaveId(Protos.SlaveID.newBuilder().setValue("slave-id"))
+        .setHostname("hostname")
+        .addAllResources(resources).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/test/java/org/apache/aurora/scheduler/state/TaskAssignerImplTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/state/TaskAssignerImplTest.java b/src/test/java/org/apache/aurora/scheduler/state/TaskAssignerImplTest.java
index 0698f78..a2df311 100644
--- a/src/test/java/org/apache/aurora/scheduler/state/TaskAssignerImplTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/state/TaskAssignerImplTest.java
@@ -27,7 +27,6 @@ import org.apache.aurora.gen.JobKey;
 import org.apache.aurora.gen.ScheduledTask;
 import org.apache.aurora.gen.TaskConfig;
 import org.apache.aurora.scheduler.HostOffer;
-import org.apache.aurora.scheduler.Resources;
 import org.apache.aurora.scheduler.TierManager;
 import org.apache.aurora.scheduler.base.TaskGroupKey;
 import org.apache.aurora.scheduler.base.Tasks;
@@ -37,6 +36,7 @@ import org.apache.aurora.scheduler.filter.SchedulingFilter.UnusedResource;
 import org.apache.aurora.scheduler.filter.SchedulingFilter.Veto;
 import org.apache.aurora.scheduler.mesos.MesosTaskFactory;
 import org.apache.aurora.scheduler.offers.OfferManager;
+import org.apache.aurora.scheduler.resources.Resources;
 import org.apache.aurora.scheduler.state.TaskAssigner.TaskAssignerImpl;
 import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 import org.apache.aurora.scheduler.storage.entities.IScheduledTask;

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/test/java/org/apache/aurora/scheduler/stats/AsyncStatsModuleTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/stats/AsyncStatsModuleTest.java b/src/test/java/org/apache/aurora/scheduler/stats/AsyncStatsModuleTest.java
index f6bbbb8..c23efed 100644
--- a/src/test/java/org/apache/aurora/scheduler/stats/AsyncStatsModuleTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/stats/AsyncStatsModuleTest.java
@@ -19,8 +19,8 @@ import org.apache.aurora.common.testing.easymock.EasyMockTest;
 import org.apache.aurora.gen.HostAttributes;
 import org.apache.aurora.gen.ResourceAggregate;
 import org.apache.aurora.scheduler.HostOffer;
-import org.apache.aurora.scheduler.ResourceType;
 import org.apache.aurora.scheduler.offers.OfferManager;
+import org.apache.aurora.scheduler.resources.ResourceType;
 import org.apache.aurora.scheduler.stats.SlotSizeCounter.MachineResource;
 import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 import org.apache.aurora.scheduler.storage.entities.IResourceAggregate;

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/test/java/org/apache/aurora/scheduler/stats/SlotSizeCounterTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/stats/SlotSizeCounterTest.java b/src/test/java/org/apache/aurora/scheduler/stats/SlotSizeCounterTest.java
index 6af1fff..bbf1097 100644
--- a/src/test/java/org/apache/aurora/scheduler/stats/SlotSizeCounterTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/stats/SlotSizeCounterTest.java
@@ -22,7 +22,7 @@ import com.google.common.collect.ImmutableMap;
 import org.apache.aurora.common.stats.StatsProvider;
 import org.apache.aurora.common.testing.easymock.EasyMockTest;
 import org.apache.aurora.gen.ResourceAggregate;
-import org.apache.aurora.scheduler.ResourceAggregates;
+import org.apache.aurora.scheduler.resources.ResourceAggregates;
 import org.apache.aurora.scheduler.stats.SlotSizeCounter.MachineResource;
 import org.apache.aurora.scheduler.stats.SlotSizeCounter.MachineResourceProvider;
 import org.apache.aurora.scheduler.storage.entities.IResourceAggregate;

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplIT.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplIT.java b/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplIT.java
index b5fd204..ff9c1d0 100644
--- a/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplIT.java
+++ b/src/test/java/org/apache/aurora/scheduler/storage/log/SnapshotStoreImplIT.java
@@ -49,9 +49,9 @@ import org.apache.aurora.gen.storage.SchedulerMetadata;
 import org.apache.aurora.gen.storage.Snapshot;
 import org.apache.aurora.gen.storage.StoredCronJob;
 import org.apache.aurora.gen.storage.StoredJobUpdateDetails;
-import org.apache.aurora.scheduler.ResourceAggregates;
 import org.apache.aurora.scheduler.base.JobKeys;
 import org.apache.aurora.scheduler.base.TaskTestUtil;
+import org.apache.aurora.scheduler.resources.ResourceAggregates;
 import org.apache.aurora.scheduler.storage.SnapshotStore;
 import org.apache.aurora.scheduler.storage.Storage;
 import org.apache.aurora.scheduler.storage.db.MigrationManager;

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/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 c3240cd..50a1fdb 100644
--- a/src/test/java/org/apache/aurora/scheduler/thrift/ReadOnlySchedulerImplTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/thrift/ReadOnlySchedulerImplTest.java
@@ -89,12 +89,12 @@ import org.junit.Before;
 import org.junit.Test;
 
 import static org.apache.aurora.gen.ResponseCode.INVALID_REQUEST;
-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.base.Numbers.convertRanges;
 import static org.apache.aurora.scheduler.base.Numbers.toRanges;
+import static org.apache.aurora.scheduler.resources.ResourceAggregates.LARGE;
+import static org.apache.aurora.scheduler.resources.ResourceAggregates.MEDIUM;
+import static org.apache.aurora.scheduler.resources.ResourceAggregates.SMALL;
+import static org.apache.aurora.scheduler.resources.ResourceAggregates.XLARGE;
 import static org.apache.aurora.scheduler.thrift.Fixtures.CRON_JOB;
 import static org.apache.aurora.scheduler.thrift.Fixtures.CRON_SCHEDULE;
 import static org.apache.aurora.scheduler.thrift.Fixtures.IDENTITY;


[2/2] aurora git commit: Moving resource-related classes into a new package

Posted by ma...@apache.org.
Moving resource-related classes into a new package

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


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

Branch: refs/heads/master
Commit: 46ce98d8c64d866cc0a94445606f0e00cfbeb649
Parents: bafdd71
Author: Maxim Khutornenko <ma...@apache.org>
Authored: Mon Apr 11 14:45:39 2016 -0700
Committer: Maxim Khutornenko <ma...@apache.org>
Committed: Mon Apr 11 14:45:39 2016 -0700

----------------------------------------------------------------------
 .../org/apache/aurora/benchmark/Offers.java     |   2 +-
 .../apache/aurora/scheduler/AcceptedOffer.java  | 234 -------------
 .../aurora/scheduler/ResourceAggregates.java    |  68 ----
 .../apache/aurora/scheduler/ResourceSlot.java   | 341 ------------------
 .../apache/aurora/scheduler/ResourceType.java   |  54 ---
 .../org/apache/aurora/scheduler/Resources.java  | 185 ----------
 .../configuration/executor/ExecutorModule.java  |   6 +-
 .../executor/ExecutorSettings.java              |   4 +-
 .../scheduler/filter/SchedulingFilter.java      |   2 +-
 .../scheduler/filter/SchedulingFilterImpl.java  |   2 +-
 .../scheduler/mesos/MesosTaskFactory.java       |   6 +-
 .../scheduler/mesos/TestExecutorSettings.java   |   4 +-
 .../scheduler/preemptor/PreemptionVictim.java   |   2 +-
 .../preemptor/PreemptionVictimFilter.java       |   6 +-
 .../aurora/scheduler/quota/QuotaManager.java    |   4 +-
 .../scheduler/resources/AcceptedOffer.java      | 235 +++++++++++++
 .../scheduler/resources/ResourceAggregates.java |  68 ++++
 .../scheduler/resources/ResourceSlot.java       | 342 +++++++++++++++++++
 .../scheduler/resources/ResourceType.java       |  54 +++
 .../aurora/scheduler/resources/Resources.java   | 186 ++++++++++
 .../apache/aurora/scheduler/sla/SlaGroup.java   |  10 +-
 .../aurora/scheduler/state/TaskAssigner.java    |   2 +-
 .../scheduler/stats/AsyncStatsModule.java       |  10 +-
 .../aurora/scheduler/stats/SlotSizeCounter.java |   2 +-
 .../aurora/scheduler/AcceptedOfferTest.java     | 304 -----------------
 .../aurora/scheduler/ResourceSlotTest.java      | 170 ---------
 .../apache/aurora/scheduler/ResourcesTest.java  | 176 ----------
 .../aurora/scheduler/app/SchedulerIT.java       |   2 +-
 .../local/simulator/ClusterSimulatorModule.java |   8 +-
 .../events/NotifyingSchedulingFilterTest.java   |   2 +-
 .../filter/SchedulingFilterImplTest.java        |   4 +-
 .../mesos/MesosTaskFactoryImplTest.java         |   8 +-
 .../apache/aurora/scheduler/mesos/Offers.java   |   8 +-
 .../aurora/scheduler/mesos/TaskExecutors.java   |   2 +-
 .../preemptor/PreemptionVictimFilterTest.java   |   4 +-
 .../scheduler/quota/QuotaManagerImplTest.java   |   2 +-
 .../scheduler/resources/AcceptedOfferTest.java  | 305 +++++++++++++++++
 .../scheduler/resources/ResourceSlotTest.java   | 170 +++++++++
 .../scheduler/resources/ResourcesTest.java      | 176 ++++++++++
 .../scheduler/state/TaskAssignerImplTest.java   |   2 +-
 .../scheduler/stats/AsyncStatsModuleTest.java   |   2 +-
 .../scheduler/stats/SlotSizeCounterTest.java    |   2 +-
 .../storage/log/SnapshotStoreImplIT.java        |   2 +-
 .../thrift/ReadOnlySchedulerImplTest.java       |   8 +-
 44 files changed, 1595 insertions(+), 1591 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/jmh/java/org/apache/aurora/benchmark/Offers.java
----------------------------------------------------------------------
diff --git a/src/jmh/java/org/apache/aurora/benchmark/Offers.java b/src/jmh/java/org/apache/aurora/benchmark/Offers.java
index 4c232f6..c22b791 100644
--- a/src/jmh/java/org/apache/aurora/benchmark/Offers.java
+++ b/src/jmh/java/org/apache/aurora/benchmark/Offers.java
@@ -20,8 +20,8 @@ import com.google.common.collect.ImmutableSet;
 import org.apache.aurora.common.quantity.Amount;
 import org.apache.aurora.common.quantity.Data;
 import org.apache.aurora.scheduler.HostOffer;
-import org.apache.aurora.scheduler.ResourceSlot;
 import org.apache.aurora.scheduler.offers.OfferManager;
+import org.apache.aurora.scheduler.resources.ResourceSlot;
 import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 import org.apache.mesos.Protos;
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/main/java/org/apache/aurora/scheduler/AcceptedOffer.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/AcceptedOffer.java b/src/main/java/org/apache/aurora/scheduler/AcceptedOffer.java
deleted file mode 100644
index 9c2dc0b..0000000
--- a/src/main/java/org/apache/aurora/scheduler/AcceptedOffer.java
+++ /dev/null
@@ -1,234 +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;
-
-import java.util.List;
-import java.util.Set;
-
-import com.google.common.base.Predicate;
-import com.google.common.base.Predicates;
-import com.google.common.collect.FluentIterable;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Sets;
-
-import org.apache.aurora.common.quantity.Data;
-import org.apache.aurora.scheduler.base.Numbers;
-import org.apache.mesos.Protos;
-import org.apache.mesos.Protos.Offer;
-import org.apache.mesos.Protos.Resource;
-
-import static java.util.Objects.requireNonNull;
-
-/**
- * Allocate resources from an accepted Mesos Offer to TaskInfo and ExecutorInfo.
- */
-public final class AcceptedOffer {
-
-  public static final String DEFAULT_ROLE_NAME = "*";
-
-  /**
-   * Reserved resource filter.
-   */
-  public static final Predicate<Resource> RESERVED =
-      e -> e.hasRole() && !e.getRole().equals(DEFAULT_ROLE_NAME);
-
-  /**
-   * Non reserved resource filter.
-   */
-  public static final Predicate<Resource> NOT_RESERVED = Predicates.not(RESERVED);
-
-  /**
-   * Helper function to check a resource value is small enough to be considered zero.
-   */
-  public static boolean nearZero(double value) {
-    return Math.abs(value) < EPSILON;
-  }
-
-  /**
-   * Get proper value for {@link org.apache.mesos.Protos.TaskInfo}'s resources.
-   * @return A list of Resource used for TaskInfo.
-   */
-  public List<Resource> getTaskResources() {
-    return taskResources;
-  }
-
-  /**
-   * Get proper value for {@link org.apache.mesos.Protos.ExecutorInfo}'s resources.
-   * @return A list of Resource used for ExecutorInfo.
-   */
-  public List<Resource> getExecutorResources() {
-    return executorResources;
-  }
-
-  /**
-   * Use this epsilon value to avoid comparison with zero.
-   */
-  private static final double EPSILON = 1e-6;
-
-  private final List<Resource> taskResources;
-  private final List<Resource> executorResources;
-
-  public static AcceptedOffer create(
-      Offer offer,
-      ResourceSlot taskSlot,
-      ResourceSlot executorSlot,
-      Set<Integer> selectedPorts,
-      TierInfo tierInfo) throws Resources.InsufficientResourcesException {
-
-    List<Resource> reservedFirst = ImmutableList.<Resource>builder()
-        .addAll(Iterables.filter(offer.getResourcesList(), RESERVED))
-        .addAll(Iterables.filter(offer.getResourcesList(), NOT_RESERVED))
-        .build();
-
-    boolean revocable = tierInfo.isRevocable();
-    List<Resource.Builder> cpuResources = filterToBuilders(
-        reservedFirst,
-        ResourceType.CPUS.getName(),
-        revocable ? Resources.REVOCABLE : Resources.NON_REVOCABLE);
-    List<Resource.Builder> memResources = filterToBuilderNonRevocable(
-        reservedFirst, ResourceType.RAM_MB.getName());
-    List<Resource.Builder> diskResources = filterToBuilderNonRevocable(
-        reservedFirst, ResourceType.DISK_MB.getName());
-    List<Resource.Builder> portsResources = filterToBuilderNonRevocable(
-        reservedFirst, ResourceType.PORTS.getName());
-
-    List<Resource> taskResources = ImmutableList.<Resource>builder()
-        .addAll(allocateScalarType(cpuResources, taskSlot.getNumCpus(), revocable))
-        .addAll(allocateScalarType(memResources, taskSlot.getRam().as(Data.MB), false))
-        .addAll(allocateScalarType(diskResources, taskSlot.getDisk().as(Data.MB), false))
-        .addAll(allocateRangeType(portsResources, selectedPorts))
-        .build();
-
-    List<Resource> executorResources = ImmutableList.<Resource>builder()
-        .addAll(allocateScalarType(cpuResources, executorSlot.getNumCpus(), revocable))
-        .addAll(allocateScalarType(memResources, executorSlot.getRam().as(Data.MB), false))
-        .addAll(allocateScalarType(diskResources, executorSlot.getDisk().as(Data.MB), false))
-        .build();
-
-    return new AcceptedOffer(taskResources, executorResources);
-  }
-
-  private AcceptedOffer(
-      List<Resource> taskResources,
-      List<Resource> executorResources) {
-
-    this.taskResources = requireNonNull(taskResources);
-    this.executorResources = requireNonNull(executorResources);
-  }
-
-  private static List<Resource> allocateRangeType(
-      List<Resource.Builder> from,
-      Set<Integer> valueSet) throws Resources.InsufficientResourcesException {
-
-    Set<Integer> leftOver = Sets.newHashSet(valueSet);
-    ImmutableList.Builder<Resource> result = ImmutableList.<Resource>builder();
-    for (Resource.Builder r : from) {
-      Set<Integer> fromResource = Sets.newHashSet(Iterables.concat(
-          Iterables.transform(r.getRanges().getRangeList(), Resources.RANGE_TO_MEMBERS)));
-      Set<Integer> available = Sets.newHashSet(Sets.intersection(leftOver, fromResource));
-      if (available.isEmpty()) {
-        continue;
-      }
-      Resource newResource = makeMesosRangeResource(r.build(), available);
-      result.add(newResource);
-      leftOver.removeAll(available);
-      if (leftOver.isEmpty()) {
-        break;
-      }
-    }
-    if (!leftOver.isEmpty()) {
-      // NOTE: this will not happen as long as Veto logic from TaskAssigner.maybeAssign is
-      // consistent.
-      // Maybe we should consider implementing resource veto with this class to ensure that.
-      throw new Resources.InsufficientResourcesException(
-          "Insufficient resource for range type when allocating from offer");
-    }
-    return result.build();
-  }
-
-  /**
-   * Creates a mesos resource of integer ranges from given prototype.
-   *
-   * @param prototype Resource prototype.
-   * @param values    Values to translate into ranges.
-   * @return A new mesos ranges resource.
-   */
-  static Resource makeMesosRangeResource(
-      Resource prototype,
-      Set<Integer> values) {
-
-    return Protos.Resource.newBuilder(prototype)
-        .setRanges(Protos.Value.Ranges.newBuilder()
-            .addAllRange(
-                Iterables.transform(Numbers.toRanges(values), ResourceSlot.RANGE_TRANSFORM)))
-        .build();
-  }
-
-  private static List<Resource> allocateScalarType(
-      List<Resource.Builder> from,
-      double amount,
-      boolean revocable) throws Resources.InsufficientResourcesException {
-
-    double remaining = amount;
-    ImmutableList.Builder<Resource> result = ImmutableList.builder();
-    for (Resource.Builder r : from) {
-      if (nearZero(remaining)) {
-        break;
-      }
-      final double available = r.getScalar().getValue();
-      if (nearZero(available)) {
-        // Skip resource slot that is already used up.
-        continue;
-      }
-      final double used = Math.min(remaining, available);
-      remaining -= used;
-      Resource.Builder newResource =
-          Resource.newBuilder(r.build())
-              .setScalar(Protos.Value.Scalar.newBuilder().setValue(used).build());
-      if (revocable) {
-        newResource.setRevocable(Resource.RevocableInfo.newBuilder());
-      }
-      result.add(newResource.build());
-      r.getScalarBuilder().setValue(available - used);
-    }
-    if (!nearZero(remaining)) {
-      // NOTE: this will not happen as long as Veto logic from TaskAssigner.maybeAssign is
-      // consistent.
-      // Maybe we should consider implementing resource veto with this class to ensure that.
-      throw new Resources.InsufficientResourcesException(
-          "Insufficient resource when allocating from offer");
-    }
-    return result.build();
-  }
-
-  private static List<Resource.Builder> filterToBuilders(
-      List<Resource> resources,
-      String name,
-      Predicate<Resource> additionalFilter) {
-
-    return FluentIterable.from(resources)
-        .filter(e -> e.getName().equals(name))
-        .filter(additionalFilter)
-        .transform(Resource::toBuilder)
-        .toList();
-  }
-
-  private static List<Resource.Builder> filterToBuilderNonRevocable(
-      List<Resource> resources,
-      String name) {
-
-    return filterToBuilders(resources, name, Resources.NON_REVOCABLE);
-  }
-}

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/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
deleted file mode 100644
index a6335c5..0000000
--- a/src/main/java/org/apache/aurora/scheduler/ResourceAggregates.java
+++ /dev/null
@@ -1,68 +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;
-
-import com.google.common.collect.Ordering;
-
-import org.apache.aurora.gen.ResourceAggregate;
-import org.apache.aurora.scheduler.storage.entities.IResourceAggregate;
-
-/**
- * Convenience class for normalizing resource measures between tasks and offers.
- */
-public final class ResourceAggregates {
-
-  public static final IResourceAggregate EMPTY =
-      IResourceAggregate.build(new ResourceAggregate(0, 0, 0));
-
-  public static final IResourceAggregate SMALL =
-      IResourceAggregate.build(new ResourceAggregate(1.0, 1024, 4096));
-
-  public static final IResourceAggregate MEDIUM =
-      IResourceAggregate.build(new ResourceAggregate(4.0, 8192, 16384));
-
-  public static final IResourceAggregate LARGE =
-      IResourceAggregate.build(new ResourceAggregate(8.0, 16384, 32768));
-
-  public static final IResourceAggregate XLARGE =
-      IResourceAggregate.build(new ResourceAggregate(16.0, 32768, 65536));
-
-  private ResourceAggregates() {
-    // Utility class.
-  }
-
-  /**
-   * a * m.
-   */
-  public static IResourceAggregate scale(IResourceAggregate a, int m) {
-    return IResourceAggregate.build(new ResourceAggregate()
-        .setNumCpus(a.getNumCpus() * m)
-        .setRamMb(a.getRamMb() * m)
-        .setDiskMb(a.getDiskMb() * m));
-  }
-
-  /**
-   * a / b.
-   * <p>
-   * This calculates how many times {@code b} "fits into" {@code a}.  Behavior is undefined when
-   * {@code b} contains resources with a value of zero.
-   */
-  public static int divide(IResourceAggregate a, IResourceAggregate b) {
-    return Ordering.natural().min(
-        a.getNumCpus() / b.getNumCpus(),
-        (double) a.getRamMb() / b.getRamMb(),
-        (double) a.getDiskMb() / b.getDiskMb()
-    ).intValue();
-  }
-}

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/main/java/org/apache/aurora/scheduler/ResourceSlot.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/ResourceSlot.java b/src/main/java/org/apache/aurora/scheduler/ResourceSlot.java
deleted file mode 100644
index 86f2667..0000000
--- a/src/main/java/org/apache/aurora/scheduler/ResourceSlot.java
+++ /dev/null
@@ -1,341 +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;
-
-import java.util.List;
-import java.util.Objects;
-import java.util.Optional;
-import java.util.Set;
-import java.util.function.Consumer;
-import java.util.function.Predicate;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Function;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Ordering;
-import com.google.common.collect.Range;
-
-import org.apache.aurora.common.quantity.Amount;
-import org.apache.aurora.common.quantity.Data;
-import org.apache.aurora.scheduler.base.Numbers;
-import org.apache.aurora.scheduler.storage.entities.ITaskConfig;
-import org.apache.mesos.Protos;
-import org.apache.mesos.Protos.ExecutorInfo;
-import org.apache.mesos.Protos.Resource;
-import org.apache.mesos.Protos.Resource.Builder;
-import org.apache.mesos.Protos.TaskInfo;
-
-import static java.util.Objects.requireNonNull;
-
-import static org.apache.aurora.common.quantity.Data.BYTES;
-import static org.apache.aurora.scheduler.ResourceType.CPUS;
-import static org.apache.aurora.scheduler.ResourceType.DISK_MB;
-import static org.apache.aurora.scheduler.ResourceType.RAM_MB;
-
-/**
- * Represents a single task/host aggregate resource vector unaware of any Mesos resource traits.
- */
-public final class ResourceSlot {
-
-  private final double numCpus;
-  private final Amount<Long, Data> disk;
-  private final Amount<Long, Data> ram;
-  private final int numPorts;
-
-  /**
-   * Empty ResourceSlot value.
-   */
-  public static final ResourceSlot NONE =
-      new ResourceSlot(0, Amount.of(0L, Data.BITS), Amount.of(0L, Data.BITS), 0);
-
-  /**
-   * Convert {@link com.google.common.collect.Range} to {@link org.apache.mesos.Protos.Value.Range}.
-   */
-  public static final Function<Range<Integer>, Protos.Value.Range> RANGE_TRANSFORM =
-      input -> Protos.Value.Range.newBuilder()
-          .setBegin(input.lowerEndpoint())
-          .setEnd(input.upperEndpoint())
-          .build();
-
-  public ResourceSlot(
-      double numCpus,
-      Amount<Long, Data> ram,
-      Amount<Long, Data> disk,
-      int numPorts) {
-
-    this.numCpus = numCpus;
-    this.ram = requireNonNull(ram);
-    this.disk = requireNonNull(disk);
-    this.numPorts = numPorts;
-  }
-
-  /**
-   * Extracts the resources required from a task.
-   *
-   * @param task Task to get resources from.
-   * @return The resources required by the task.
-   */
-  public static ResourceSlot from(ITaskConfig task) {
-    requireNonNull(task);
-    return new ResourceSlot(
-        task.getNumCpus(),
-        Amount.of(task.getRamMb(), Data.MB),
-        Amount.of(task.getDiskMb(), Data.MB),
-        task.getRequestedPorts().size());
-  }
-
-  /**
-   * Ensures that the revocable setting on the executor and task CPU resources match.
-   *
-   * @param task Task to check for resource type alignment.
-   * @return A possibly-modified task, with aligned CPU resource types.
-   */
-  public static TaskInfo matchResourceTypes(TaskInfo task) {
-    TaskInfo.Builder taskBuilder = task.toBuilder();
-
-    Optional<Resource> revocableTaskCpu = taskBuilder.getResourcesList().stream()
-        .filter(r -> r.getName().equals(CPUS.getName()))
-        .filter(Resource::hasRevocable)
-        .findFirst();
-    ExecutorInfo.Builder executorBuilder = taskBuilder.getExecutorBuilder();
-
-    Consumer<Builder> matchRevocable = builder -> {
-      if (revocableTaskCpu.isPresent()) {
-        builder.setRevocable(revocableTaskCpu.get().getRevocable());
-      } else {
-        builder.clearRevocable();
-      }
-    };
-
-    executorBuilder.getResourcesBuilderList().stream()
-        .filter(r -> r.getName().equals(CPUS.getName()))
-        .forEach(matchRevocable);
-
-    return taskBuilder.build();
-  }
-
-  /**
-   * Convenience method for adapting to Mesos resources without applying a port range.
-   *
-   * @param tierInfo Task tier info.
-   * @return Mesos resources.
-   */
-  public List<Protos.Resource> toResourceList(TierInfo tierInfo) {
-    return ImmutableList.<Protos.Resource>builder()
-        .add(makeMesosResource(CPUS, numCpus, tierInfo.isRevocable()))
-        .add(makeMesosResource(DISK_MB, disk.as(Data.MB), false))
-        .add(makeMesosResource(RAM_MB, ram.as(Data.MB), false))
-        .build();
-  }
-
-  /**
-   * Creates a mesos resource of integer ranges.
-   *
-   * @param resourceType Resource type.
-   * @param values    Values to translate into ranges.
-   * @return A new mesos ranges resource.
-   */
-  @VisibleForTesting
-  public static Protos.Resource makeMesosRangeResource(
-      ResourceType resourceType,
-      Set<Integer> values) {
-
-    return Protos.Resource.newBuilder()
-        .setName(resourceType.getName())
-        .setType(Protos.Value.Type.RANGES)
-        .setRanges(Protos.Value.Ranges.newBuilder()
-            .addAllRange(Iterables.transform(Numbers.toRanges(values), RANGE_TRANSFORM)))
-        .build();
-  }
-
-  /**
-   * Creates a scalar mesos resource.
-   *
-   * @param resourceType Resource type.
-   * @param value Value for the resource.
-   * @param revocable Flag indicating if this resource is revocable.
-   * @return A mesos resource.
-   */
-  @VisibleForTesting
-  static Protos.Resource makeMesosResource(
-      ResourceType resourceType,
-      double value,
-      boolean revocable) {
-
-    Protos.Resource.Builder builder = Protos.Resource.newBuilder()
-        .setName(resourceType.getName())
-        .setType(Protos.Value.Type.SCALAR)
-        .setScalar(Protos.Value.Scalar.newBuilder().setValue(value));
-
-    if (revocable) {
-      builder.setRevocable(Protos.Resource.RevocableInfo.newBuilder());
-    }
-
-    return builder.build();
-  }
-
-  /**
-   * Generates a ResourceSlot where each resource component is a max out of the two components.
-   *
-   * @param a A resource to compare.
-   * @param b A resource to compare.
-   *
-   * @return Returns a ResourceSlot instance where each component is a max of the two components.
-   */
-  @VisibleForTesting
-  static ResourceSlot maxElements(ResourceSlot a, ResourceSlot b) {
-    double maxCPU = Math.max(a.getNumCpus(), b.getNumCpus());
-    Amount<Long, Data> maxRAM = Amount.of(
-        Math.max(a.getRam().as(Data.MB), b.getRam().as(Data.MB)),
-        Data.MB);
-    Amount<Long, Data> maxDisk = Amount.of(
-        Math.max(a.getDisk().as(Data.MB), b.getDisk().as(Data.MB)),
-        Data.MB);
-    int maxPorts = Math.max(a.getNumPorts(), b.getNumPorts());
-
-    return new ResourceSlot(maxCPU, maxRAM, maxDisk, maxPorts);
-  }
-
-  /**
-   * Number of CPUs.
-   *
-   * @return CPUs.
-   */
-  public double getNumCpus() {
-    return numCpus;
-  }
-
-  /**
-   * Disk amount.
-   *
-   * @return Disk.
-   */
-  public Amount<Long, Data> getDisk() {
-    return disk;
-  }
-
-  /**
-   * RAM amount.
-   *
-   * @return RAM.
-   */
-  public Amount<Long, Data> getRam() {
-    return ram;
-  }
-
-  /**
-   * Number of ports.
-   *
-   * @return Port count.
-   */
-  public int getNumPorts() {
-    return numPorts;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (!(o instanceof ResourceSlot)) {
-      return false;
-    }
-
-    ResourceSlot other = (ResourceSlot) o;
-    return Objects.equals(numCpus, other.numCpus)
-        && Objects.equals(ram, other.ram)
-        && Objects.equals(disk, other.disk)
-        && Objects.equals(numPorts, other.numPorts);
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(numCpus, ram, disk, numPorts);
-  }
-
-  /**
-   * Sums up all resources in {@code slots}.
-   *
-   * @param slots Resource slots to sum up.
-   * @return Sum of all resource slots.
-   */
-  public static ResourceSlot sum(Iterable<ResourceSlot> slots) {
-    ResourceSlot sum = NONE;
-
-    for (ResourceSlot r : slots) {
-      sum = sum.add(r);
-    }
-
-    return sum;
-  }
-
-  /**
-   * Adds {@code other}.
-   *
-   * @param other Resource slot to add.
-   * @return Result.
-   */
-  public ResourceSlot add(ResourceSlot other) {
-    return new ResourceSlot(
-        getNumCpus() + other.getNumCpus(),
-        Amount.of(getRam().as(BYTES) + other.getRam().as(BYTES), BYTES),
-        Amount.of(getDisk().as(BYTES) + other.getDisk().as(BYTES), BYTES),
-        getNumPorts() + other.getNumPorts());
-  }
-
-  /**
-   * Subtracts {@code other}.
-   *
-   * @param other Resource slot to subtract.
-   * @return Result.
-   */
-  public ResourceSlot subtract(ResourceSlot other) {
-    return new ResourceSlot(
-        getNumCpus() - other.getNumCpus(),
-        Amount.of(getRam().as(BYTES) - other.getRam().as(BYTES), BYTES),
-        Amount.of(getDisk().as(BYTES) - other.getDisk().as(BYTES), BYTES),
-        getNumPorts() - other.getNumPorts());
-  }
-
-  /**
-   * A Resources object is greater than another iff _all_ of its resource components are greater
-   * or equal. A Resources object compares as equal if some but not all components are greater than
-   * or equal to the other.
-   */
-  public static final Ordering<ResourceSlot> ORDER = new Ordering<ResourceSlot>() {
-    @Override
-    public int compare(ResourceSlot left, ResourceSlot right) {
-      int diskC = left.getDisk().compareTo(right.getDisk());
-      int ramC = left.getRam().compareTo(right.getRam());
-      int portC = Integer.compare(left.getNumPorts(), right.getNumPorts());
-      int cpuC = Double.compare(left.getNumCpus(), right.getNumCpus());
-
-      List<Integer> vector = ImmutableList.of(diskC, ramC, portC, cpuC);
-
-      if (vector.stream().allMatch(IS_ZERO))  {
-        return 0;
-      }
-
-      if (vector.stream().filter(IS_ZERO.negate()).allMatch(e -> e > 0)) {
-        return 1;
-      }
-
-      if (vector.stream().filter(IS_ZERO.negate()).allMatch(e -> e < 0)) {
-        return -1;
-      }
-
-      return 0;
-    }
-  };
-
-  private static final Predicate<Integer> IS_ZERO = e -> e == 0;
-}

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/main/java/org/apache/aurora/scheduler/ResourceType.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/ResourceType.java b/src/main/java/org/apache/aurora/scheduler/ResourceType.java
deleted file mode 100644
index b4efc8d..0000000
--- a/src/main/java/org/apache/aurora/scheduler/ResourceType.java
+++ /dev/null
@@ -1,54 +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;
-
-import com.google.common.annotations.VisibleForTesting;
-
-import static java.util.Objects.requireNonNull;
-
-/**
- * Defines Mesos resource types.
- */
-@VisibleForTesting
-public enum ResourceType {
-  /**
-   * CPU resource.
-   */
-  CPUS("cpus"),
-
-  /**
-   * RAM resource.
-   */
-  RAM_MB("mem"),
-
-  /**
-   * DISK resource.
-   */
-  DISK_MB("disk"),
-
-  /**
-   * Port resource.
-   */
-  PORTS("ports");
-
-  private final String resourceName;
-
-  ResourceType(String resourceName) {
-    this.resourceName = requireNonNull(resourceName);
-  }
-
-  public String getName() {
-    return resourceName;
-  }
-}

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/main/java/org/apache/aurora/scheduler/Resources.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/Resources.java b/src/main/java/org/apache/aurora/scheduler/Resources.java
deleted file mode 100644
index 4baf9dd..0000000
--- a/src/main/java/org/apache/aurora/scheduler/Resources.java
+++ /dev/null
@@ -1,185 +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;
-
-import java.util.Collections;
-import java.util.List;
-import java.util.Set;
-
-import com.google.common.base.Function;
-import com.google.common.base.Predicate;
-import com.google.common.base.Predicates;
-import com.google.common.collect.ContiguousSet;
-import com.google.common.collect.DiscreteDomain;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-
-import org.apache.aurora.common.quantity.Amount;
-import org.apache.aurora.common.quantity.Data;
-import org.apache.mesos.Protos.Offer;
-import org.apache.mesos.Protos.Resource;
-import org.apache.mesos.Protos.Value.Range;
-
-import static java.util.Objects.requireNonNull;
-
-import static org.apache.aurora.scheduler.ResourceType.CPUS;
-import static org.apache.aurora.scheduler.ResourceType.DISK_MB;
-import static org.apache.aurora.scheduler.ResourceType.PORTS;
-import static org.apache.aurora.scheduler.ResourceType.RAM_MB;
-
-/**
- * A container for multiple Mesos resource vectors.
- */
-public final class Resources {
-
-  /**
-   * CPU resource filter.
-   */
-  private static final Predicate<Resource> CPU = e -> e.getName().equals(CPUS.getName());
-
-  /**
-   * Revocable resource filter.
-   */
-  public static final Predicate<Resource> REVOCABLE =
-      Predicates.or(Predicates.not(CPU), Predicates.and(CPU, Resource::hasRevocable));
-
-  /**
-   * Non-revocable resource filter.
-   */
-  public static final Predicate<Resource> NON_REVOCABLE = Predicates.not(Resource::hasRevocable);
-
-  /**
-   * Convert range to set of integers.
-   */
-  public static final Function<Range, Set<Integer>> RANGE_TO_MEMBERS =
-      range -> ContiguousSet.create(
-          com.google.common.collect.Range.closed((int) range.getBegin(), (int) range.getEnd()),
-          DiscreteDomain.integers());
-
-  private final Iterable<Resource> mesosResources;
-
-  private Resources(Iterable<Resource> mesosResources) {
-    this.mesosResources = ImmutableList.copyOf(mesosResources);
-  }
-
-  /**
-   * Extracts the resources available in a slave offer.
-   *
-   * @param offer Offer to get resources from.
-   * @return The resources available in the offer.
-   */
-  public static Resources from(Offer offer) {
-    return new Resources(requireNonNull(offer.getResourcesList()));
-  }
-
-  /**
-   * Filters resources by the provided {@code predicate}.
-   *
-   * @param predicate Predicate filter.
-   * @return A new {@code Resources} object containing only filtered Mesos resources.
-   */
-  public Resources filter(Predicate<Resource> predicate) {
-    return new Resources(Iterables.filter(mesosResources, predicate));
-  }
-
-  /**
-   * Filters resources using the provided {@code tierInfo} instance.
-   *
-   * @param tierInfo Tier info.
-   * @return A new {@code Resources} object containing only filtered Mesos resources.
-   */
-  public Resources filter(TierInfo tierInfo) {
-    return filter(tierInfo.isRevocable() ? REVOCABLE : NON_REVOCABLE);
-  }
-
-  /**
-   * Gets generalized aggregated resource view.
-   *
-   * @return {@code ResourceSlot} instance.
-   */
-  public ResourceSlot slot() {
-    return new ResourceSlot(getScalarValue(CPUS.getName()),
-        Amount.of((long) getScalarValue(RAM_MB.getName()), Data.MB),
-        Amount.of((long) getScalarValue(DISK_MB.getName()), Data.MB),
-        getNumAvailablePorts());
-  }
-
-  /**
-   * Attempts to grab {@code numPorts} from this resource instance.
-   *
-   * @param numPorts The number of ports to grab.
-   * @return The set of ports grabbed.
-   * @throws InsufficientResourcesException if not enough ports were available.
-   */
-  public Set<Integer> getPorts(int numPorts)
-      throws InsufficientResourcesException {
-
-    if (numPorts == 0) {
-      return ImmutableSet.of();
-    }
-
-    List<Integer> availablePorts = Lists.newArrayList(Sets.newHashSet(Iterables.concat(
-        Iterables.transform(getPortRanges(), RANGE_TO_MEMBERS))));
-
-    if (availablePorts.size() < numPorts) {
-      throw new InsufficientResourcesException(
-          String.format("Could not get %d ports from %s", numPorts, availablePorts));
-    }
-
-    Collections.shuffle(availablePorts);
-    return ImmutableSet.copyOf(availablePorts.subList(0, numPorts));
-  }
-
-  private int getNumAvailablePorts() {
-    int offeredPorts = 0;
-    for (Range range : getPortRanges()) {
-      offeredPorts += 1 + range.getEnd() - range.getBegin();
-    }
-    return offeredPorts;
-  }
-
-  private double getScalarValue(String key) {
-    Iterable<Resource> resources = getResources(key);
-    double value = 0;
-    for (Resource r : resources) {
-      value += r.getScalar().getValue();
-    }
-    return value;
-  }
-
-  private Iterable<Resource> getResources(String key) {
-    return Iterables.filter(mesosResources, e -> e.getName().equals(key));
-  }
-
-  private Iterable<Range> getPortRanges() {
-    ImmutableList.Builder<Range> ranges = ImmutableList.builder();
-    for (Resource r : getResources(PORTS.getName())) {
-      ranges.addAll(r.getRanges().getRangeList().iterator());
-    }
-
-    return ranges.build();
-  }
-
-  /**
-   * Thrown when there are insufficient resources to satisfy a request.
-   */
-  public static class InsufficientResourcesException extends RuntimeException {
-    InsufficientResourcesException(String message) {
-      super(message);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/main/java/org/apache/aurora/scheduler/configuration/executor/ExecutorModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/configuration/executor/ExecutorModule.java b/src/main/java/org/apache/aurora/scheduler/configuration/executor/ExecutorModule.java
index 1fe27a5..dd9e12b 100644
--- a/src/main/java/org/apache/aurora/scheduler/configuration/executor/ExecutorModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/configuration/executor/ExecutorModule.java
@@ -35,7 +35,7 @@ import org.apache.aurora.common.base.MorePreconditions;
 import org.apache.aurora.common.quantity.Amount;
 import org.apache.aurora.common.quantity.Data;
 import org.apache.aurora.gen.Volume;
-import org.apache.aurora.scheduler.ResourceType;
+import org.apache.aurora.scheduler.resources.ResourceType;
 import org.apache.mesos.Protos;
 import org.apache.mesos.Protos.CommandInfo;
 import org.apache.mesos.Protos.CommandInfo.URI;
@@ -44,8 +44,8 @@ import org.apache.mesos.Protos.Resource;
 import org.apache.mesos.Protos.Value.Scalar;
 import org.apache.mesos.Protos.Value.Type;
 
-import static org.apache.aurora.scheduler.ResourceType.CPUS;
-import static org.apache.aurora.scheduler.ResourceType.RAM_MB;
+import static org.apache.aurora.scheduler.resources.ResourceType.CPUS;
+import static org.apache.aurora.scheduler.resources.ResourceType.RAM_MB;
 
 /**
  * Binding module for {@link ExecutorSettings}.

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/main/java/org/apache/aurora/scheduler/configuration/executor/ExecutorSettings.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/configuration/executor/ExecutorSettings.java b/src/main/java/org/apache/aurora/scheduler/configuration/executor/ExecutorSettings.java
index e4279b1..78e7be9 100644
--- a/src/main/java/org/apache/aurora/scheduler/configuration/executor/ExecutorSettings.java
+++ b/src/main/java/org/apache/aurora/scheduler/configuration/executor/ExecutorSettings.java
@@ -17,8 +17,8 @@ import java.util.Objects;
 
 import org.apache.aurora.common.quantity.Amount;
 import org.apache.aurora.common.quantity.Data;
-import org.apache.aurora.scheduler.ResourceSlot;
-import org.apache.aurora.scheduler.ResourceType;
+import org.apache.aurora.scheduler.resources.ResourceSlot;
+import org.apache.aurora.scheduler.resources.ResourceType;
 
 import static java.util.Objects.requireNonNull;
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/main/java/org/apache/aurora/scheduler/filter/SchedulingFilter.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/filter/SchedulingFilter.java b/src/main/java/org/apache/aurora/scheduler/filter/SchedulingFilter.java
index 1e8eb0c..625e6d5 100644
--- a/src/main/java/org/apache/aurora/scheduler/filter/SchedulingFilter.java
+++ b/src/main/java/org/apache/aurora/scheduler/filter/SchedulingFilter.java
@@ -18,7 +18,7 @@ import java.util.Set;
 
 import com.google.common.base.MoreObjects;
 
-import org.apache.aurora.scheduler.ResourceSlot;
+import org.apache.aurora.scheduler.resources.ResourceSlot;
 import org.apache.aurora.scheduler.storage.entities.IConstraint;
 import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 import org.apache.aurora.scheduler.storage.entities.ITaskConfig;

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/main/java/org/apache/aurora/scheduler/filter/SchedulingFilterImpl.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/filter/SchedulingFilterImpl.java b/src/main/java/org/apache/aurora/scheduler/filter/SchedulingFilterImpl.java
index e9ee049..f8c57f9 100644
--- a/src/main/java/org/apache/aurora/scheduler/filter/SchedulingFilterImpl.java
+++ b/src/main/java/org/apache/aurora/scheduler/filter/SchedulingFilterImpl.java
@@ -29,9 +29,9 @@ import org.apache.aurora.common.quantity.Amount;
 import org.apache.aurora.common.quantity.Data;
 import org.apache.aurora.gen.MaintenanceMode;
 import org.apache.aurora.gen.TaskConstraint;
-import org.apache.aurora.scheduler.ResourceSlot;
 import org.apache.aurora.scheduler.configuration.ConfigurationManager;
 import org.apache.aurora.scheduler.configuration.executor.ExecutorSettings;
+import org.apache.aurora.scheduler.resources.ResourceSlot;
 import org.apache.aurora.scheduler.storage.entities.IAttribute;
 import org.apache.aurora.scheduler.storage.entities.IConstraint;
 import org.apache.aurora.scheduler.storage.entities.IHostAttributes;

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/main/java/org/apache/aurora/scheduler/mesos/MesosTaskFactory.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/mesos/MesosTaskFactory.java b/src/main/java/org/apache/aurora/scheduler/mesos/MesosTaskFactory.java
index fb7c7b2..b325106 100644
--- a/src/main/java/org/apache/aurora/scheduler/mesos/MesosTaskFactory.java
+++ b/src/main/java/org/apache/aurora/scheduler/mesos/MesosTaskFactory.java
@@ -27,14 +27,14 @@ import com.google.protobuf.ByteString;
 import org.apache.aurora.GuavaUtils;
 import org.apache.aurora.Protobufs;
 import org.apache.aurora.codec.ThriftBinaryCodec;
-import org.apache.aurora.scheduler.AcceptedOffer;
-import org.apache.aurora.scheduler.ResourceSlot;
-import org.apache.aurora.scheduler.Resources;
 import org.apache.aurora.scheduler.TierManager;
 import org.apache.aurora.scheduler.base.JobKeys;
 import org.apache.aurora.scheduler.base.SchedulerException;
 import org.apache.aurora.scheduler.base.Tasks;
 import org.apache.aurora.scheduler.configuration.executor.ExecutorSettings;
+import org.apache.aurora.scheduler.resources.AcceptedOffer;
+import org.apache.aurora.scheduler.resources.ResourceSlot;
+import org.apache.aurora.scheduler.resources.Resources;
 import org.apache.aurora.scheduler.storage.entities.IAssignedTask;
 import org.apache.aurora.scheduler.storage.entities.IDockerContainer;
 import org.apache.aurora.scheduler.storage.entities.IJobKey;

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/main/java/org/apache/aurora/scheduler/mesos/TestExecutorSettings.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/mesos/TestExecutorSettings.java b/src/main/java/org/apache/aurora/scheduler/mesos/TestExecutorSettings.java
index 8cef410..1d252bb 100644
--- a/src/main/java/org/apache/aurora/scheduler/mesos/TestExecutorSettings.java
+++ b/src/main/java/org/apache/aurora/scheduler/mesos/TestExecutorSettings.java
@@ -15,12 +15,12 @@ package org.apache.aurora.scheduler.mesos;
 
 import com.google.common.collect.ImmutableList;
 
-import org.apache.aurora.scheduler.ResourceSlot;
-import org.apache.aurora.scheduler.ResourceType;
 import org.apache.aurora.scheduler.base.TaskTestUtil;
 import org.apache.aurora.scheduler.configuration.executor.ExecutorConfig;
 import org.apache.aurora.scheduler.configuration.executor.ExecutorSettings;
 import org.apache.aurora.scheduler.configuration.executor.Executors;
+import org.apache.aurora.scheduler.resources.ResourceSlot;
+import org.apache.aurora.scheduler.resources.ResourceType;
 import org.apache.mesos.Protos.CommandInfo;
 import org.apache.mesos.Protos.CommandInfo.URI;
 import org.apache.mesos.Protos.ExecutorInfo;

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/main/java/org/apache/aurora/scheduler/preemptor/PreemptionVictim.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/preemptor/PreemptionVictim.java b/src/main/java/org/apache/aurora/scheduler/preemptor/PreemptionVictim.java
index 7d92843..98be997 100644
--- a/src/main/java/org/apache/aurora/scheduler/preemptor/PreemptionVictim.java
+++ b/src/main/java/org/apache/aurora/scheduler/preemptor/PreemptionVictim.java
@@ -17,7 +17,7 @@ import java.util.Objects;
 
 import com.google.common.base.MoreObjects;
 
-import org.apache.aurora.scheduler.ResourceSlot;
+import org.apache.aurora.scheduler.resources.ResourceSlot;
 import org.apache.aurora.scheduler.storage.entities.IAssignedTask;
 import org.apache.aurora.scheduler.storage.entities.ITaskConfig;
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/main/java/org/apache/aurora/scheduler/preemptor/PreemptionVictimFilter.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/preemptor/PreemptionVictimFilter.java b/src/main/java/org/apache/aurora/scheduler/preemptor/PreemptionVictimFilter.java
index d3b7963..9a37ee7 100644
--- a/src/main/java/org/apache/aurora/scheduler/preemptor/PreemptionVictimFilter.java
+++ b/src/main/java/org/apache/aurora/scheduler/preemptor/PreemptionVictimFilter.java
@@ -28,8 +28,6 @@ import com.google.common.collect.Ordering;
 import com.google.common.collect.Sets;
 
 import org.apache.aurora.scheduler.HostOffer;
-import org.apache.aurora.scheduler.ResourceSlot;
-import org.apache.aurora.scheduler.Resources;
 import org.apache.aurora.scheduler.TierManager;
 import org.apache.aurora.scheduler.configuration.executor.ExecutorSettings;
 import org.apache.aurora.scheduler.filter.AttributeAggregate;
@@ -37,13 +35,15 @@ import org.apache.aurora.scheduler.filter.SchedulingFilter;
 import org.apache.aurora.scheduler.filter.SchedulingFilter.ResourceRequest;
 import org.apache.aurora.scheduler.filter.SchedulingFilter.UnusedResource;
 import org.apache.aurora.scheduler.filter.SchedulingFilter.Veto;
+import org.apache.aurora.scheduler.resources.ResourceSlot;
+import org.apache.aurora.scheduler.resources.Resources;
 import org.apache.aurora.scheduler.storage.Storage.StoreProvider;
 import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
 import org.apache.aurora.scheduler.storage.entities.ITaskConfig;
 
 import static java.util.Objects.requireNonNull;
 
-import static org.apache.aurora.scheduler.ResourceSlot.sum;
+import static org.apache.aurora.scheduler.resources.ResourceSlot.sum;
 
 /**
  * Filters active tasks (victims) and available offer (slack) resources that can accommodate a

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/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 c18836a..bf476aa 100644
--- a/src/main/java/org/apache/aurora/scheduler/quota/QuotaManager.java
+++ b/src/main/java/org/apache/aurora/scheduler/quota/QuotaManager.java
@@ -31,11 +31,11 @@ import com.google.common.collect.RangeSet;
 
 import org.apache.aurora.gen.JobUpdateQuery;
 import org.apache.aurora.gen.ResourceAggregate;
-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.resources.ResourceAggregates;
 import org.apache.aurora.scheduler.storage.JobUpdateStore;
 import org.apache.aurora.scheduler.storage.Storage.MutableStoreProvider;
 import org.apache.aurora.scheduler.storage.Storage.StoreProvider;
@@ -62,8 +62,8 @@ 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.resources.ResourceAggregates.EMPTY;
 import static org.apache.aurora.scheduler.updater.Updates.getInstanceIds;
 
 /**

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/main/java/org/apache/aurora/scheduler/resources/AcceptedOffer.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/resources/AcceptedOffer.java b/src/main/java/org/apache/aurora/scheduler/resources/AcceptedOffer.java
new file mode 100644
index 0000000..6a5237f
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/resources/AcceptedOffer.java
@@ -0,0 +1,235 @@
+/**
+ * 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.resources;
+
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Predicate;
+import com.google.common.base.Predicates;
+import com.google.common.collect.FluentIterable;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Sets;
+
+import org.apache.aurora.common.quantity.Data;
+import org.apache.aurora.scheduler.TierInfo;
+import org.apache.aurora.scheduler.base.Numbers;
+import org.apache.mesos.Protos;
+import org.apache.mesos.Protos.Offer;
+import org.apache.mesos.Protos.Resource;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * Allocate resources from an accepted Mesos Offer to TaskInfo and ExecutorInfo.
+ */
+public final class AcceptedOffer {
+
+  public static final String DEFAULT_ROLE_NAME = "*";
+
+  /**
+   * Reserved resource filter.
+   */
+  public static final Predicate<Resource> RESERVED =
+      e -> e.hasRole() && !e.getRole().equals(DEFAULT_ROLE_NAME);
+
+  /**
+   * Non reserved resource filter.
+   */
+  public static final Predicate<Resource> NOT_RESERVED = Predicates.not(RESERVED);
+
+  /**
+   * Helper function to check a resource value is small enough to be considered zero.
+   */
+  public static boolean nearZero(double value) {
+    return Math.abs(value) < EPSILON;
+  }
+
+  /**
+   * Get proper value for {@link org.apache.mesos.Protos.TaskInfo}'s resources.
+   * @return A list of Resource used for TaskInfo.
+   */
+  public List<Resource> getTaskResources() {
+    return taskResources;
+  }
+
+  /**
+   * Get proper value for {@link org.apache.mesos.Protos.ExecutorInfo}'s resources.
+   * @return A list of Resource used for ExecutorInfo.
+   */
+  public List<Resource> getExecutorResources() {
+    return executorResources;
+  }
+
+  /**
+   * Use this epsilon value to avoid comparison with zero.
+   */
+  private static final double EPSILON = 1e-6;
+
+  private final List<Resource> taskResources;
+  private final List<Resource> executorResources;
+
+  public static AcceptedOffer create(
+      Offer offer,
+      ResourceSlot taskSlot,
+      ResourceSlot executorSlot,
+      Set<Integer> selectedPorts,
+      TierInfo tierInfo) throws Resources.InsufficientResourcesException {
+
+    List<Resource> reservedFirst = ImmutableList.<Resource>builder()
+        .addAll(Iterables.filter(offer.getResourcesList(), RESERVED))
+        .addAll(Iterables.filter(offer.getResourcesList(), NOT_RESERVED))
+        .build();
+
+    boolean revocable = tierInfo.isRevocable();
+    List<Resource.Builder> cpuResources = filterToBuilders(
+        reservedFirst,
+        ResourceType.CPUS.getName(),
+        revocable ? Resources.REVOCABLE : Resources.NON_REVOCABLE);
+    List<Resource.Builder> memResources = filterToBuilderNonRevocable(
+        reservedFirst, ResourceType.RAM_MB.getName());
+    List<Resource.Builder> diskResources = filterToBuilderNonRevocable(
+        reservedFirst, ResourceType.DISK_MB.getName());
+    List<Resource.Builder> portsResources = filterToBuilderNonRevocable(
+        reservedFirst, ResourceType.PORTS.getName());
+
+    List<Resource> taskResources = ImmutableList.<Resource>builder()
+        .addAll(allocateScalarType(cpuResources, taskSlot.getNumCpus(), revocable))
+        .addAll(allocateScalarType(memResources, taskSlot.getRam().as(Data.MB), false))
+        .addAll(allocateScalarType(diskResources, taskSlot.getDisk().as(Data.MB), false))
+        .addAll(allocateRangeType(portsResources, selectedPorts))
+        .build();
+
+    List<Resource> executorResources = ImmutableList.<Resource>builder()
+        .addAll(allocateScalarType(cpuResources, executorSlot.getNumCpus(), revocable))
+        .addAll(allocateScalarType(memResources, executorSlot.getRam().as(Data.MB), false))
+        .addAll(allocateScalarType(diskResources, executorSlot.getDisk().as(Data.MB), false))
+        .build();
+
+    return new AcceptedOffer(taskResources, executorResources);
+  }
+
+  private AcceptedOffer(
+      List<Resource> taskResources,
+      List<Resource> executorResources) {
+
+    this.taskResources = requireNonNull(taskResources);
+    this.executorResources = requireNonNull(executorResources);
+  }
+
+  private static List<Resource> allocateRangeType(
+      List<Resource.Builder> from,
+      Set<Integer> valueSet) throws Resources.InsufficientResourcesException {
+
+    Set<Integer> leftOver = Sets.newHashSet(valueSet);
+    ImmutableList.Builder<Resource> result = ImmutableList.<Resource>builder();
+    for (Resource.Builder r : from) {
+      Set<Integer> fromResource = Sets.newHashSet(Iterables.concat(
+          Iterables.transform(r.getRanges().getRangeList(), Resources.RANGE_TO_MEMBERS)));
+      Set<Integer> available = Sets.newHashSet(Sets.intersection(leftOver, fromResource));
+      if (available.isEmpty()) {
+        continue;
+      }
+      Resource newResource = makeMesosRangeResource(r.build(), available);
+      result.add(newResource);
+      leftOver.removeAll(available);
+      if (leftOver.isEmpty()) {
+        break;
+      }
+    }
+    if (!leftOver.isEmpty()) {
+      // NOTE: this will not happen as long as Veto logic from TaskAssigner.maybeAssign is
+      // consistent.
+      // Maybe we should consider implementing resource veto with this class to ensure that.
+      throw new Resources.InsufficientResourcesException(
+          "Insufficient resource for range type when allocating from offer");
+    }
+    return result.build();
+  }
+
+  /**
+   * Creates a mesos resource of integer ranges from given prototype.
+   *
+   * @param prototype Resource prototype.
+   * @param values    Values to translate into ranges.
+   * @return A new mesos ranges resource.
+   */
+  static Resource makeMesosRangeResource(
+      Resource prototype,
+      Set<Integer> values) {
+
+    return Protos.Resource.newBuilder(prototype)
+        .setRanges(Protos.Value.Ranges.newBuilder()
+            .addAllRange(
+                Iterables.transform(Numbers.toRanges(values), ResourceSlot.RANGE_TRANSFORM)))
+        .build();
+  }
+
+  private static List<Resource> allocateScalarType(
+      List<Resource.Builder> from,
+      double amount,
+      boolean revocable) throws Resources.InsufficientResourcesException {
+
+    double remaining = amount;
+    ImmutableList.Builder<Resource> result = ImmutableList.builder();
+    for (Resource.Builder r : from) {
+      if (nearZero(remaining)) {
+        break;
+      }
+      final double available = r.getScalar().getValue();
+      if (nearZero(available)) {
+        // Skip resource slot that is already used up.
+        continue;
+      }
+      final double used = Math.min(remaining, available);
+      remaining -= used;
+      Resource.Builder newResource =
+          Resource.newBuilder(r.build())
+              .setScalar(Protos.Value.Scalar.newBuilder().setValue(used).build());
+      if (revocable) {
+        newResource.setRevocable(Resource.RevocableInfo.newBuilder());
+      }
+      result.add(newResource.build());
+      r.getScalarBuilder().setValue(available - used);
+    }
+    if (!nearZero(remaining)) {
+      // NOTE: this will not happen as long as Veto logic from TaskAssigner.maybeAssign is
+      // consistent.
+      // Maybe we should consider implementing resource veto with this class to ensure that.
+      throw new Resources.InsufficientResourcesException(
+          "Insufficient resource when allocating from offer");
+    }
+    return result.build();
+  }
+
+  private static List<Resource.Builder> filterToBuilders(
+      List<Resource> resources,
+      String name,
+      Predicate<Resource> additionalFilter) {
+
+    return FluentIterable.from(resources)
+        .filter(e -> e.getName().equals(name))
+        .filter(additionalFilter)
+        .transform(Resource::toBuilder)
+        .toList();
+  }
+
+  private static List<Resource.Builder> filterToBuilderNonRevocable(
+      List<Resource> resources,
+      String name) {
+
+    return filterToBuilders(resources, name, Resources.NON_REVOCABLE);
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/main/java/org/apache/aurora/scheduler/resources/ResourceAggregates.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/resources/ResourceAggregates.java b/src/main/java/org/apache/aurora/scheduler/resources/ResourceAggregates.java
new file mode 100644
index 0000000..302eb87
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/resources/ResourceAggregates.java
@@ -0,0 +1,68 @@
+/**
+ * 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.resources;
+
+import com.google.common.collect.Ordering;
+
+import org.apache.aurora.gen.ResourceAggregate;
+import org.apache.aurora.scheduler.storage.entities.IResourceAggregate;
+
+/**
+ * Convenience class for normalizing resource measures between tasks and offers.
+ */
+public final class ResourceAggregates {
+
+  public static final IResourceAggregate EMPTY =
+      IResourceAggregate.build(new ResourceAggregate(0, 0, 0));
+
+  public static final IResourceAggregate SMALL =
+      IResourceAggregate.build(new ResourceAggregate(1.0, 1024, 4096));
+
+  public static final IResourceAggregate MEDIUM =
+      IResourceAggregate.build(new ResourceAggregate(4.0, 8192, 16384));
+
+  public static final IResourceAggregate LARGE =
+      IResourceAggregate.build(new ResourceAggregate(8.0, 16384, 32768));
+
+  public static final IResourceAggregate XLARGE =
+      IResourceAggregate.build(new ResourceAggregate(16.0, 32768, 65536));
+
+  private ResourceAggregates() {
+    // Utility class.
+  }
+
+  /**
+   * a * m.
+   */
+  public static IResourceAggregate scale(IResourceAggregate a, int m) {
+    return IResourceAggregate.build(new ResourceAggregate()
+        .setNumCpus(a.getNumCpus() * m)
+        .setRamMb(a.getRamMb() * m)
+        .setDiskMb(a.getDiskMb() * m));
+  }
+
+  /**
+   * a / b.
+   * <p>
+   * This calculates how many times {@code b} "fits into" {@code a}.  Behavior is undefined when
+   * {@code b} contains resources with a value of zero.
+   */
+  public static int divide(IResourceAggregate a, IResourceAggregate b) {
+    return Ordering.natural().min(
+        a.getNumCpus() / b.getNumCpus(),
+        (double) a.getRamMb() / b.getRamMb(),
+        (double) a.getDiskMb() / b.getDiskMb()
+    ).intValue();
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/main/java/org/apache/aurora/scheduler/resources/ResourceSlot.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/resources/ResourceSlot.java b/src/main/java/org/apache/aurora/scheduler/resources/ResourceSlot.java
new file mode 100644
index 0000000..43696d2
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/resources/ResourceSlot.java
@@ -0,0 +1,342 @@
+/**
+ * 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.resources;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.function.Predicate;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Function;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Ordering;
+import com.google.common.collect.Range;
+
+import org.apache.aurora.common.quantity.Amount;
+import org.apache.aurora.common.quantity.Data;
+import org.apache.aurora.scheduler.TierInfo;
+import org.apache.aurora.scheduler.base.Numbers;
+import org.apache.aurora.scheduler.storage.entities.ITaskConfig;
+import org.apache.mesos.Protos;
+import org.apache.mesos.Protos.ExecutorInfo;
+import org.apache.mesos.Protos.Resource;
+import org.apache.mesos.Protos.Resource.Builder;
+import org.apache.mesos.Protos.TaskInfo;
+
+import static java.util.Objects.requireNonNull;
+
+import static org.apache.aurora.common.quantity.Data.BYTES;
+import static org.apache.aurora.scheduler.resources.ResourceType.CPUS;
+import static org.apache.aurora.scheduler.resources.ResourceType.DISK_MB;
+import static org.apache.aurora.scheduler.resources.ResourceType.RAM_MB;
+
+/**
+ * Represents a single task/host aggregate resource vector unaware of any Mesos resource traits.
+ */
+public final class ResourceSlot {
+
+  private final double numCpus;
+  private final Amount<Long, Data> disk;
+  private final Amount<Long, Data> ram;
+  private final int numPorts;
+
+  /**
+   * Empty ResourceSlot value.
+   */
+  public static final ResourceSlot NONE =
+      new ResourceSlot(0, Amount.of(0L, Data.BITS), Amount.of(0L, Data.BITS), 0);
+
+  /**
+   * Convert {@link com.google.common.collect.Range} to {@link org.apache.mesos.Protos.Value.Range}.
+   */
+  public static final Function<Range<Integer>, Protos.Value.Range> RANGE_TRANSFORM =
+      input -> Protos.Value.Range.newBuilder()
+          .setBegin(input.lowerEndpoint())
+          .setEnd(input.upperEndpoint())
+          .build();
+
+  public ResourceSlot(
+      double numCpus,
+      Amount<Long, Data> ram,
+      Amount<Long, Data> disk,
+      int numPorts) {
+
+    this.numCpus = numCpus;
+    this.ram = requireNonNull(ram);
+    this.disk = requireNonNull(disk);
+    this.numPorts = numPorts;
+  }
+
+  /**
+   * Extracts the resources required from a task.
+   *
+   * @param task Task to get resources from.
+   * @return The resources required by the task.
+   */
+  public static ResourceSlot from(ITaskConfig task) {
+    requireNonNull(task);
+    return new ResourceSlot(
+        task.getNumCpus(),
+        Amount.of(task.getRamMb(), Data.MB),
+        Amount.of(task.getDiskMb(), Data.MB),
+        task.getRequestedPorts().size());
+  }
+
+  /**
+   * Ensures that the revocable setting on the executor and task CPU resources match.
+   *
+   * @param task Task to check for resource type alignment.
+   * @return A possibly-modified task, with aligned CPU resource types.
+   */
+  public static TaskInfo matchResourceTypes(TaskInfo task) {
+    TaskInfo.Builder taskBuilder = task.toBuilder();
+
+    Optional<Resource> revocableTaskCpu = taskBuilder.getResourcesList().stream()
+        .filter(r -> r.getName().equals(CPUS.getName()))
+        .filter(Resource::hasRevocable)
+        .findFirst();
+    ExecutorInfo.Builder executorBuilder = taskBuilder.getExecutorBuilder();
+
+    Consumer<Builder> matchRevocable = builder -> {
+      if (revocableTaskCpu.isPresent()) {
+        builder.setRevocable(revocableTaskCpu.get().getRevocable());
+      } else {
+        builder.clearRevocable();
+      }
+    };
+
+    executorBuilder.getResourcesBuilderList().stream()
+        .filter(r -> r.getName().equals(CPUS.getName()))
+        .forEach(matchRevocable);
+
+    return taskBuilder.build();
+  }
+
+  /**
+   * Convenience method for adapting to Mesos resources without applying a port range.
+   *
+   * @param tierInfo Task tier info.
+   * @return Mesos resources.
+   */
+  public List<Protos.Resource> toResourceList(TierInfo tierInfo) {
+    return ImmutableList.<Protos.Resource>builder()
+        .add(makeMesosResource(CPUS, numCpus, tierInfo.isRevocable()))
+        .add(makeMesosResource(DISK_MB, disk.as(Data.MB), false))
+        .add(makeMesosResource(RAM_MB, ram.as(Data.MB), false))
+        .build();
+  }
+
+  /**
+   * Creates a mesos resource of integer ranges.
+   *
+   * @param resourceType Resource type.
+   * @param values    Values to translate into ranges.
+   * @return A new mesos ranges resource.
+   */
+  @VisibleForTesting
+  public static Protos.Resource makeMesosRangeResource(
+      ResourceType resourceType,
+      Set<Integer> values) {
+
+    return Protos.Resource.newBuilder()
+        .setName(resourceType.getName())
+        .setType(Protos.Value.Type.RANGES)
+        .setRanges(Protos.Value.Ranges.newBuilder()
+            .addAllRange(Iterables.transform(Numbers.toRanges(values), RANGE_TRANSFORM)))
+        .build();
+  }
+
+  /**
+   * Creates a scalar mesos resource.
+   *
+   * @param resourceType Resource type.
+   * @param value Value for the resource.
+   * @param revocable Flag indicating if this resource is revocable.
+   * @return A mesos resource.
+   */
+  @VisibleForTesting
+  static Protos.Resource makeMesosResource(
+      ResourceType resourceType,
+      double value,
+      boolean revocable) {
+
+    Protos.Resource.Builder builder = Protos.Resource.newBuilder()
+        .setName(resourceType.getName())
+        .setType(Protos.Value.Type.SCALAR)
+        .setScalar(Protos.Value.Scalar.newBuilder().setValue(value));
+
+    if (revocable) {
+      builder.setRevocable(Protos.Resource.RevocableInfo.newBuilder());
+    }
+
+    return builder.build();
+  }
+
+  /**
+   * Generates a ResourceSlot where each resource component is a max out of the two components.
+   *
+   * @param a A resource to compare.
+   * @param b A resource to compare.
+   *
+   * @return Returns a ResourceSlot instance where each component is a max of the two components.
+   */
+  @VisibleForTesting
+  static ResourceSlot maxElements(ResourceSlot a, ResourceSlot b) {
+    double maxCPU = Math.max(a.getNumCpus(), b.getNumCpus());
+    Amount<Long, Data> maxRAM = Amount.of(
+        Math.max(a.getRam().as(Data.MB), b.getRam().as(Data.MB)),
+        Data.MB);
+    Amount<Long, Data> maxDisk = Amount.of(
+        Math.max(a.getDisk().as(Data.MB), b.getDisk().as(Data.MB)),
+        Data.MB);
+    int maxPorts = Math.max(a.getNumPorts(), b.getNumPorts());
+
+    return new ResourceSlot(maxCPU, maxRAM, maxDisk, maxPorts);
+  }
+
+  /**
+   * Number of CPUs.
+   *
+   * @return CPUs.
+   */
+  public double getNumCpus() {
+    return numCpus;
+  }
+
+  /**
+   * Disk amount.
+   *
+   * @return Disk.
+   */
+  public Amount<Long, Data> getDisk() {
+    return disk;
+  }
+
+  /**
+   * RAM amount.
+   *
+   * @return RAM.
+   */
+  public Amount<Long, Data> getRam() {
+    return ram;
+  }
+
+  /**
+   * Number of ports.
+   *
+   * @return Port count.
+   */
+  public int getNumPorts() {
+    return numPorts;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (!(o instanceof ResourceSlot)) {
+      return false;
+    }
+
+    ResourceSlot other = (ResourceSlot) o;
+    return Objects.equals(numCpus, other.numCpus)
+        && Objects.equals(ram, other.ram)
+        && Objects.equals(disk, other.disk)
+        && Objects.equals(numPorts, other.numPorts);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(numCpus, ram, disk, numPorts);
+  }
+
+  /**
+   * Sums up all resources in {@code slots}.
+   *
+   * @param slots Resource slots to sum up.
+   * @return Sum of all resource slots.
+   */
+  public static ResourceSlot sum(Iterable<ResourceSlot> slots) {
+    ResourceSlot sum = NONE;
+
+    for (ResourceSlot r : slots) {
+      sum = sum.add(r);
+    }
+
+    return sum;
+  }
+
+  /**
+   * Adds {@code other}.
+   *
+   * @param other Resource slot to add.
+   * @return Result.
+   */
+  public ResourceSlot add(ResourceSlot other) {
+    return new ResourceSlot(
+        getNumCpus() + other.getNumCpus(),
+        Amount.of(getRam().as(BYTES) + other.getRam().as(BYTES), BYTES),
+        Amount.of(getDisk().as(BYTES) + other.getDisk().as(BYTES), BYTES),
+        getNumPorts() + other.getNumPorts());
+  }
+
+  /**
+   * Subtracts {@code other}.
+   *
+   * @param other Resource slot to subtract.
+   * @return Result.
+   */
+  public ResourceSlot subtract(ResourceSlot other) {
+    return new ResourceSlot(
+        getNumCpus() - other.getNumCpus(),
+        Amount.of(getRam().as(BYTES) - other.getRam().as(BYTES), BYTES),
+        Amount.of(getDisk().as(BYTES) - other.getDisk().as(BYTES), BYTES),
+        getNumPorts() - other.getNumPorts());
+  }
+
+  /**
+   * A Resources object is greater than another iff _all_ of its resource components are greater
+   * or equal. A Resources object compares as equal if some but not all components are greater than
+   * or equal to the other.
+   */
+  public static final Ordering<ResourceSlot> ORDER = new Ordering<ResourceSlot>() {
+    @Override
+    public int compare(ResourceSlot left, ResourceSlot right) {
+      int diskC = left.getDisk().compareTo(right.getDisk());
+      int ramC = left.getRam().compareTo(right.getRam());
+      int portC = Integer.compare(left.getNumPorts(), right.getNumPorts());
+      int cpuC = Double.compare(left.getNumCpus(), right.getNumCpus());
+
+      List<Integer> vector = ImmutableList.of(diskC, ramC, portC, cpuC);
+
+      if (vector.stream().allMatch(IS_ZERO))  {
+        return 0;
+      }
+
+      if (vector.stream().filter(IS_ZERO.negate()).allMatch(e -> e > 0)) {
+        return 1;
+      }
+
+      if (vector.stream().filter(IS_ZERO.negate()).allMatch(e -> e < 0)) {
+        return -1;
+      }
+
+      return 0;
+    }
+  };
+
+  private static final Predicate<Integer> IS_ZERO = e -> e == 0;
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/main/java/org/apache/aurora/scheduler/resources/ResourceType.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/resources/ResourceType.java b/src/main/java/org/apache/aurora/scheduler/resources/ResourceType.java
new file mode 100644
index 0000000..5900ccb
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/resources/ResourceType.java
@@ -0,0 +1,54 @@
+/**
+ * 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.resources;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * Defines Mesos resource types.
+ */
+@VisibleForTesting
+public enum ResourceType {
+  /**
+   * CPU resource.
+   */
+  CPUS("cpus"),
+
+  /**
+   * RAM resource.
+   */
+  RAM_MB("mem"),
+
+  /**
+   * DISK resource.
+   */
+  DISK_MB("disk"),
+
+  /**
+   * Port resource.
+   */
+  PORTS("ports");
+
+  private final String resourceName;
+
+  ResourceType(String resourceName) {
+    this.resourceName = requireNonNull(resourceName);
+  }
+
+  public String getName() {
+    return resourceName;
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/main/java/org/apache/aurora/scheduler/resources/Resources.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/resources/Resources.java b/src/main/java/org/apache/aurora/scheduler/resources/Resources.java
new file mode 100644
index 0000000..46b31f0
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/resources/Resources.java
@@ -0,0 +1,186 @@
+/**
+ * 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.resources;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Function;
+import com.google.common.base.Predicate;
+import com.google.common.base.Predicates;
+import com.google.common.collect.ContiguousSet;
+import com.google.common.collect.DiscreteDomain;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+
+import org.apache.aurora.common.quantity.Amount;
+import org.apache.aurora.common.quantity.Data;
+import org.apache.aurora.scheduler.TierInfo;
+import org.apache.mesos.Protos.Offer;
+import org.apache.mesos.Protos.Resource;
+import org.apache.mesos.Protos.Value.Range;
+
+import static java.util.Objects.requireNonNull;
+
+import static org.apache.aurora.scheduler.resources.ResourceType.CPUS;
+import static org.apache.aurora.scheduler.resources.ResourceType.DISK_MB;
+import static org.apache.aurora.scheduler.resources.ResourceType.PORTS;
+import static org.apache.aurora.scheduler.resources.ResourceType.RAM_MB;
+
+/**
+ * A container for multiple Mesos resource vectors.
+ */
+public final class Resources {
+
+  /**
+   * CPU resource filter.
+   */
+  private static final Predicate<Resource> CPU = e -> e.getName().equals(CPUS.getName());
+
+  /**
+   * Revocable resource filter.
+   */
+  public static final Predicate<Resource> REVOCABLE =
+      Predicates.or(Predicates.not(CPU), Predicates.and(CPU, Resource::hasRevocable));
+
+  /**
+   * Non-revocable resource filter.
+   */
+  public static final Predicate<Resource> NON_REVOCABLE = Predicates.not(Resource::hasRevocable);
+
+  /**
+   * Convert range to set of integers.
+   */
+  public static final Function<Range, Set<Integer>> RANGE_TO_MEMBERS =
+      range -> ContiguousSet.create(
+          com.google.common.collect.Range.closed((int) range.getBegin(), (int) range.getEnd()),
+          DiscreteDomain.integers());
+
+  private final Iterable<Resource> mesosResources;
+
+  private Resources(Iterable<Resource> mesosResources) {
+    this.mesosResources = ImmutableList.copyOf(mesosResources);
+  }
+
+  /**
+   * Extracts the resources available in a slave offer.
+   *
+   * @param offer Offer to get resources from.
+   * @return The resources available in the offer.
+   */
+  public static Resources from(Offer offer) {
+    return new Resources(requireNonNull(offer.getResourcesList()));
+  }
+
+  /**
+   * Filters resources by the provided {@code predicate}.
+   *
+   * @param predicate Predicate filter.
+   * @return A new {@code Resources} object containing only filtered Mesos resources.
+   */
+  public Resources filter(Predicate<Resource> predicate) {
+    return new Resources(Iterables.filter(mesosResources, predicate));
+  }
+
+  /**
+   * Filters resources using the provided {@code tierInfo} instance.
+   *
+   * @param tierInfo Tier info.
+   * @return A new {@code Resources} object containing only filtered Mesos resources.
+   */
+  public Resources filter(TierInfo tierInfo) {
+    return filter(tierInfo.isRevocable() ? REVOCABLE : NON_REVOCABLE);
+  }
+
+  /**
+   * Gets generalized aggregated resource view.
+   *
+   * @return {@code ResourceSlot} instance.
+   */
+  public ResourceSlot slot() {
+    return new ResourceSlot(getScalarValue(CPUS.getName()),
+        Amount.of((long) getScalarValue(RAM_MB.getName()), Data.MB),
+        Amount.of((long) getScalarValue(DISK_MB.getName()), Data.MB),
+        getNumAvailablePorts());
+  }
+
+  /**
+   * Attempts to grab {@code numPorts} from this resource instance.
+   *
+   * @param numPorts The number of ports to grab.
+   * @return The set of ports grabbed.
+   * @throws InsufficientResourcesException if not enough ports were available.
+   */
+  public Set<Integer> getPorts(int numPorts)
+      throws InsufficientResourcesException {
+
+    if (numPorts == 0) {
+      return ImmutableSet.of();
+    }
+
+    List<Integer> availablePorts = Lists.newArrayList(Sets.newHashSet(Iterables.concat(
+        Iterables.transform(getPortRanges(), RANGE_TO_MEMBERS))));
+
+    if (availablePorts.size() < numPorts) {
+      throw new InsufficientResourcesException(
+          String.format("Could not get %d ports from %s", numPorts, availablePorts));
+    }
+
+    Collections.shuffle(availablePorts);
+    return ImmutableSet.copyOf(availablePorts.subList(0, numPorts));
+  }
+
+  private int getNumAvailablePorts() {
+    int offeredPorts = 0;
+    for (Range range : getPortRanges()) {
+      offeredPorts += 1 + range.getEnd() - range.getBegin();
+    }
+    return offeredPorts;
+  }
+
+  private double getScalarValue(String key) {
+    Iterable<Resource> resources = getResources(key);
+    double value = 0;
+    for (Resource r : resources) {
+      value += r.getScalar().getValue();
+    }
+    return value;
+  }
+
+  private Iterable<Resource> getResources(String key) {
+    return Iterables.filter(mesosResources, e -> e.getName().equals(key));
+  }
+
+  private Iterable<Range> getPortRanges() {
+    ImmutableList.Builder<Range> ranges = ImmutableList.builder();
+    for (Resource r : getResources(PORTS.getName())) {
+      ranges.addAll(r.getRanges().getRangeList().iterator());
+    }
+
+    return ranges.build();
+  }
+
+  /**
+   * Thrown when there are insufficient resources to satisfy a request.
+   */
+  public static class InsufficientResourcesException extends RuntimeException {
+    InsufficientResourcesException(String message) {
+      super(message);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/main/java/org/apache/aurora/scheduler/sla/SlaGroup.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/sla/SlaGroup.java b/src/main/java/org/apache/aurora/scheduler/sla/SlaGroup.java
index bf7c084..2c044a6 100644
--- a/src/main/java/org/apache/aurora/scheduler/sla/SlaGroup.java
+++ b/src/main/java/org/apache/aurora/scheduler/sla/SlaGroup.java
@@ -29,11 +29,11 @@ import org.apache.aurora.scheduler.base.Tasks;
 import org.apache.aurora.scheduler.storage.entities.IJobKey;
 import org.apache.aurora.scheduler.storage.entities.IScheduledTask;
 
-import static org.apache.aurora.scheduler.ResourceAggregates.EMPTY;
-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.resources.ResourceAggregates.EMPTY;
+import static org.apache.aurora.scheduler.resources.ResourceAggregates.LARGE;
+import static org.apache.aurora.scheduler.resources.ResourceAggregates.MEDIUM;
+import static org.apache.aurora.scheduler.resources.ResourceAggregates.SMALL;
+import static org.apache.aurora.scheduler.resources.ResourceAggregates.XLARGE;
 
 /**
  * Defines a logical grouping criteria to be applied over a set of tasks.

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/main/java/org/apache/aurora/scheduler/state/TaskAssigner.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/state/TaskAssigner.java b/src/main/java/org/apache/aurora/scheduler/state/TaskAssigner.java
index 0c467a6..7d43d4a 100644
--- a/src/main/java/org/apache/aurora/scheduler/state/TaskAssigner.java
+++ b/src/main/java/org/apache/aurora/scheduler/state/TaskAssigner.java
@@ -28,7 +28,6 @@ import com.google.common.collect.FluentIterable;
 import org.apache.aurora.common.inject.TimedInterceptor.Timed;
 import org.apache.aurora.common.stats.Stats;
 import org.apache.aurora.scheduler.HostOffer;
-import org.apache.aurora.scheduler.Resources;
 import org.apache.aurora.scheduler.TierInfo;
 import org.apache.aurora.scheduler.TierManager;
 import org.apache.aurora.scheduler.base.TaskGroupKey;
@@ -39,6 +38,7 @@ import org.apache.aurora.scheduler.filter.SchedulingFilter.Veto;
 import org.apache.aurora.scheduler.filter.SchedulingFilter.VetoGroup;
 import org.apache.aurora.scheduler.mesos.MesosTaskFactory;
 import org.apache.aurora.scheduler.offers.OfferManager;
+import org.apache.aurora.scheduler.resources.Resources;
 import org.apache.aurora.scheduler.storage.entities.IAssignedTask;
 import org.apache.mesos.Protos.TaskInfo;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/main/java/org/apache/aurora/scheduler/stats/AsyncStatsModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/stats/AsyncStatsModule.java b/src/main/java/org/apache/aurora/scheduler/stats/AsyncStatsModule.java
index 08eb6d6..03dfa27 100644
--- a/src/main/java/org/apache/aurora/scheduler/stats/AsyncStatsModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/stats/AsyncStatsModule.java
@@ -29,20 +29,20 @@ import org.apache.aurora.common.quantity.Data;
 import org.apache.aurora.common.quantity.Time;
 import org.apache.aurora.gen.ResourceAggregate;
 import org.apache.aurora.scheduler.HostOffer;
-import org.apache.aurora.scheduler.ResourceSlot;
-import org.apache.aurora.scheduler.Resources;
 import org.apache.aurora.scheduler.SchedulerServicesModule;
 import org.apache.aurora.scheduler.base.Conversions;
 import org.apache.aurora.scheduler.offers.OfferManager;
+import org.apache.aurora.scheduler.resources.ResourceSlot;
+import org.apache.aurora.scheduler.resources.Resources;
 import org.apache.aurora.scheduler.stats.SlotSizeCounter.MachineResource;
 import org.apache.aurora.scheduler.stats.SlotSizeCounter.MachineResourceProvider;
 import org.apache.aurora.scheduler.storage.entities.IResourceAggregate;
 
 import static java.util.Objects.requireNonNull;
 
-import static org.apache.aurora.scheduler.ResourceSlot.NONE;
-import static org.apache.aurora.scheduler.Resources.NON_REVOCABLE;
-import static org.apache.aurora.scheduler.Resources.REVOCABLE;
+import static org.apache.aurora.scheduler.resources.ResourceSlot.NONE;
+import static org.apache.aurora.scheduler.resources.Resources.NON_REVOCABLE;
+import static org.apache.aurora.scheduler.resources.Resources.REVOCABLE;
 
 /**
  * Module to configure export of cluster-wide resource allocation and consumption statistics.

http://git-wip-us.apache.org/repos/asf/aurora/blob/46ce98d8/src/main/java/org/apache/aurora/scheduler/stats/SlotSizeCounter.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/stats/SlotSizeCounter.java b/src/main/java/org/apache/aurora/scheduler/stats/SlotSizeCounter.java
index c9e57ec..1f71b00 100644
--- a/src/main/java/org/apache/aurora/scheduler/stats/SlotSizeCounter.java
+++ b/src/main/java/org/apache/aurora/scheduler/stats/SlotSizeCounter.java
@@ -25,7 +25,7 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableMultimap;
 
-import org.apache.aurora.scheduler.ResourceAggregates;
+import org.apache.aurora.scheduler.resources.ResourceAggregates;
 import org.apache.aurora.scheduler.storage.entities.IResourceAggregate;
 
 import static java.util.Objects.requireNonNull;