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/05/25 20:21:32 UTC

aurora git commit: Generalizing resource management on the client

Repository: aurora
Updated Branches:
  refs/heads/master 310180fcd -> d521dcd72


Generalizing resource management on the client

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


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

Branch: refs/heads/master
Commit: d521dcd72166a2f10ba4548f5cf5b769d708a888
Parents: 310180f
Author: Maxim Khutornenko <ma...@apache.org>
Authored: Wed May 25 13:21:23 2016 -0700
Committer: Maxim Khutornenko <ma...@apache.org>
Committed: Wed May 25 13:21:23 2016 -0700

----------------------------------------------------------------------
 3rdparty/python/requirements.txt                |   1 +
 src/main/python/apache/aurora/admin/admin.py    |  27 +++--
 .../python/apache/aurora/client/cli/jobs.py     |  11 +-
 .../python/apache/aurora/client/cli/quota.py    |  18 +--
 src/main/python/apache/aurora/config/BUILD      |   1 +
 .../python/apache/aurora/config/resource.py     | 112 +++++++++++++++++++
 .../apache/aurora/client/cli/test_quota.py      |  12 +-
 .../apache/aurora/client/cli/test_status.py     |  24 ++--
 .../apache/aurora/config/test_resources.py      |  60 ++++++++++
 9 files changed, 224 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/d521dcd7/3rdparty/python/requirements.txt
----------------------------------------------------------------------
diff --git a/3rdparty/python/requirements.txt b/3rdparty/python/requirements.txt
index eab0acc..5cc3cb1 100644
--- a/3rdparty/python/requirements.txt
+++ b/3rdparty/python/requirements.txt
@@ -15,6 +15,7 @@
 
 bottle==0.11.6
 CherryPy==3.6.0
+enum34==1.1.6
 mako==0.4.0
 mock==1.0.1
 mox==0.5.3

http://git-wip-us.apache.org/repos/asf/aurora/blob/d521dcd7/src/main/python/apache/aurora/admin/admin.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/admin/admin.py b/src/main/python/apache/aurora/admin/admin.py
index e309460..76009b9 100644
--- a/src/main/python/apache/aurora/admin/admin.py
+++ b/src/main/python/apache/aurora/admin/admin.py
@@ -19,7 +19,7 @@ import optparse
 import sys
 
 from twitter.common import app, log
-from twitter.common.quantity import Amount, Data, Time
+from twitter.common.quantity import Data, Time
 from twitter.common.quantity.parse_simple import parse_data, parse_time
 
 from apache.aurora.client.api import AuroraClientAPI
@@ -36,6 +36,7 @@ from apache.aurora.client.base import (
 from apache.aurora.common.aurora_job_key import AuroraJobKey
 from apache.aurora.common.clusters import CLUSTERS
 from apache.aurora.common.shellify import shellify
+from apache.aurora.config.resource import ResourceManager, ResourceType
 
 from .admin_util import (
     FILENAME_OPTION,
@@ -209,18 +210,26 @@ def increase_quota(cluster, role, cpu_str, ram_str, disk_str):
   Increases the amount of production quota allocated to a user.
   """
   cpu = float(cpu_str)
-  ram = parse_data(ram_str)
-  disk = parse_data(disk_str)
+  ram = parse_data(ram_str).as_(Data.MB)
+  disk = parse_data(disk_str).as_(Data.MB)
 
   client = make_admin_client(cluster)
   resp = client.get_quota(role)
   quota = resp.result.getQuotaResult.quota
-  log.info('Current quota for %s:\n\tCPU\t%s\n\tRAM\t%s MB\n\tDisk\t%s MB' %
-           (role, quota.numCpus, quota.ramMb, quota.diskMb))
-
-  new_cpu = float(cpu + quota.numCpus)
-  new_ram = int((ram + Amount(quota.ramMb, Data.MB)).as_(Data.MB))
-  new_disk = int((disk + Amount(quota.diskMb, Data.MB)).as_(Data.MB))
+  resource_details = ResourceManager.resource_details_from_quota(quota)
+  log.info('Current quota for %s:\n\t%s' % (
+      role,
+      '\n\t'.join('%s\t%s%s' % (
+          r.resource_type.display_name,
+          r.value,
+          r.resource_type.display_unit) for r in resource_details)))
+
+  new_cpu = ResourceType.CPUS.value_type(
+    cpu + ResourceManager.quantity_of(resource_details, ResourceType.CPUS))
+  new_ram = ResourceType.RAM_MB.value_type(
+    ram + ResourceManager.quantity_of(resource_details, ResourceType.RAM_MB))
+  new_disk = ResourceType.DISK_MB.value_type(
+    disk + ResourceManager.quantity_of(resource_details, ResourceType.DISK_MB))
 
   log.info('Attempting to update quota for %s to\n\tCPU\t%s\n\tRAM\t%s MB\n\tDisk\t%s MB' %
            (role, new_cpu, new_ram, new_disk))

http://git-wip-us.apache.org/repos/asf/aurora/blob/d521dcd7/src/main/python/apache/aurora/client/cli/jobs.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/jobs.py b/src/main/python/apache/aurora/client/cli/jobs.py
index e8bc38a..336d6fa 100644
--- a/src/main/python/apache/aurora/client/cli/jobs.py
+++ b/src/main/python/apache/aurora/client/cli/jobs.py
@@ -67,6 +67,7 @@ from apache.aurora.client.cli.options import (
     CommandOption
 )
 from apache.aurora.common.aurora_job_key import AuroraJobKey
+from apache.aurora.config.resource import ResourceManager
 
 from gen.apache.aurora.api.constants import ACTIVE_STATES, AURORA_EXECUTOR_NAME
 from gen.apache.aurora.api.ttypes import ExecutorConfig, ResponseCode, ScheduleStatus
@@ -716,11 +717,15 @@ class StatusCommand(Verb):
               ScheduleStatus._VALUES_TO_NAMES[scheduled_task.status],
               assigned_task.slaveHost))
 
+      resource_details = ResourceManager.resource_details_from_task(task_info)
       if task_info:
-        task_strings.append("""\t  cpus: %s, ram: %s MB, disk: %s MB""" % (
-            task_info.numCpus, task_info.ramMb, task_info.diskMb))
+        task_strings.append("""\t  %s""" % ", ".join("%s: %s%s" % (
+            r.resource_type.display_name,
+            r.value,
+            r.resource_type.display_unit) for r in resource_details))
+
       if assigned_task.assignedPorts:
-        task_strings.append("\t  ports: %s" % assigned_task.assignedPorts)
+        task_strings.append("\t  assigned ports: %s" % assigned_task.assignedPorts)
         # TODO(mchucarroll): only add the max if taskInfo is filled in!
         task_strings.append("\t  failure count: %s (max %s)" % (scheduled_task.failureCount,
             task_info.maxTaskFailures))

http://git-wip-us.apache.org/repos/asf/aurora/blob/d521dcd7/src/main/python/apache/aurora/client/cli/quota.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/quota.py b/src/main/python/apache/aurora/client/cli/quota.py
index 43d9810..8c4420b 100644
--- a/src/main/python/apache/aurora/client/cli/quota.py
+++ b/src/main/python/apache/aurora/client/cli/quota.py
@@ -20,6 +20,7 @@ from thrift.TSerialization import serialize
 from apache.aurora.client.cli import EXIT_OK, Noun, Verb
 from apache.aurora.client.cli.context import AuroraCommandContext
 from apache.aurora.client.cli.options import JSON_WRITE_OPTION, ROLE_ARGUMENT
+from apache.aurora.config.resource import ResourceManager
 
 
 class GetQuotaCmd(Verb):
@@ -35,19 +36,12 @@ class GetQuotaCmd(Verb):
     return [JSON_WRITE_OPTION, ROLE_ARGUMENT]
 
   def render_quota(self, write_json, quota_resp):
-    def get_quota_json(quota):
-      result = {}
-      result['cpu'] = quota.numCpus
-      result['ram'] = float(quota.ramMb) / 1024
-      result['disk'] = float(quota.diskMb) / 1024
-      return result
-
     def get_quota_str(quota):
-      result = []
-      result.append('  CPU: %s' % quota.numCpus)
-      result.append('  RAM: %f GB' % (float(quota.ramMb) / 1024))
-      result.append('  Disk: %f GB' % (float(quota.diskMb) / 1024))
-      return result
+      resource_details = ResourceManager.resource_details_from_quota(quota)
+      return ('  %s: %s%s' % (
+          r.resource_type.display_name,
+          r.value,
+          r.resource_type.display_unit) for r in resource_details)
 
     if write_json:
       return serialize(quota_resp.result.getQuotaResult,

http://git-wip-us.apache.org/repos/asf/aurora/blob/d521dcd7/src/main/python/apache/aurora/config/BUILD
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/config/BUILD b/src/main/python/apache/aurora/config/BUILD
index 8c763c3..12e7fe9 100644
--- a/src/main/python/apache/aurora/config/BUILD
+++ b/src/main/python/apache/aurora/config/BUILD
@@ -18,6 +18,7 @@ python_library(
   name = 'config',
   sources = rglobs('*.py'),
   dependencies = [
+    '3rdparty/python:enum34',
     '3rdparty/python:pystachio',
     '3rdparty/python:twitter.common.lang',
     'api/src/main/thrift/org/apache/aurora/gen',

http://git-wip-us.apache.org/repos/asf/aurora/blob/d521dcd7/src/main/python/apache/aurora/config/resource.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/config/resource.py b/src/main/python/apache/aurora/config/resource.py
new file mode 100644
index 0000000..85e1d00
--- /dev/null
+++ b/src/main/python/apache/aurora/config/resource.py
@@ -0,0 +1,112 @@
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+from collections import namedtuple
+from numbers import Number
+
+from enum import Enum, unique
+
+from gen.apache.aurora.api.ttypes import Resource
+
+ResourceDetails = namedtuple('ResourceDetails', ['resource_type', 'value'])
+
+
+@unique
+class ResourceType(Enum):
+  """Describes Aurora resource types and their traits."""
+
+  CPUS = ('numCpus', 'CPU', ' core(s)', float, 1)
+  RAM_MB = ('ramMb', 'RAM', ' MB', int, 2)
+  DISK_MB = ('diskMb', 'Disk', ' MB', int, 3)
+  PORTS = ('namedPort', 'Port', '', str, 4)
+
+  def __init__(self, field, display_name, display_unit, value_type, display_position):
+    self._field = field
+    self._display_name = display_name
+    self._display_unit = display_unit
+    self._value_type = value_type
+    self._display_position = display_position
+
+  @property
+  def field(self):
+    return self._field
+
+  @property
+  def display_name(self):
+    return self._display_name
+
+  @property
+  def display_unit(self):
+    return self._display_unit
+
+  @property
+  def value_type(self):
+    return self._value_type
+
+  @property
+  def display_position(self):
+    return self._display_position
+
+  def resource_value(self, resource):
+    return resource.__dict__.get(self._field)
+
+  @classmethod
+  def from_resource(cls, resource):
+    for _, member in cls.__members__.items():
+      if resource.__dict__.get(member.field) is not None:
+        return member
+    else:
+      raise ValueError("Unknown resource: %s" % resource)
+
+
+class ResourceManager(object):
+  """Provides helper methods for working with Aurora resources."""
+
+  @classmethod
+  def resource_details(cls, resources):
+    result = []
+    if resources:
+      for resource in list(resources):
+        r_type = ResourceType.from_resource(resource)
+        result.append(ResourceDetails(r_type, r_type.resource_value(resource)))
+      return sorted(result, key=lambda rd: rd.resource_type.display_position)
+    return result
+
+  @classmethod
+  def resource_details_from_quota(cls, quota):
+    return cls.resource_details(cls._backfill_resources(quota))
+
+  @classmethod
+  def resource_details_from_task(cls, task):
+    return cls.resource_details(cls._backfill_resources(task))
+
+  @classmethod
+  def quantity_of(cls, resource_details, resource_type):
+    result = 0.0
+    for d in resource_details:
+      if d.resource_type is resource_type:
+        result += d.value if isinstance(d.value, Number) else 1
+    return result
+
+  @classmethod
+  def _backfill_resources(cls, r_object):
+    resources = list(r_object.resources) if r_object.resources else None
+    if resources is None:
+      resources = [
+          Resource(numCpus=r_object.numCpus),
+          Resource(ramMb=r_object.ramMb),
+          Resource(diskMb=r_object.diskMb)
+      ]
+      if hasattr(r_object, 'requestedPorts'):
+        resources += [Resource(namedPort=p) for p in r_object.requestedPorts or []]
+    return resources

http://git-wip-us.apache.org/repos/asf/aurora/blob/d521dcd7/src/test/python/apache/aurora/client/cli/test_quota.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/test_quota.py b/src/test/python/apache/aurora/client/cli/test_quota.py
index e2d7f42..b566296 100644
--- a/src/test/python/apache/aurora/client/cli/test_quota.py
+++ b/src/test/python/apache/aurora/client/cli/test_quota.py
@@ -51,19 +51,19 @@ class TestGetQuotaCommand(AuroraClientCommandTest):
     api.get_quota.return_value = response
 
   def test_get_quota_no_consumption(self):
-    assert ('Allocated:\n  CPU: 5\n  RAM: 20.000000 GB\n  Disk: 40.000000 GB' ==
+    assert ('Allocated:\n  CPU: 5 core(s)\n  RAM: 20480 MB\n  Disk: 40960 MB' ==
             self._get_quota(False, ['quota', 'get', 'west/bozo']))
 
   def test_get_quota_with_consumption(self):
-    expected_output = ('Allocated:\n  CPU: 5\n  RAM: 20.000000 GB\n  Disk: 40.000000 GB\n'
+    expected_output = ('Allocated:\n  CPU: 5 core(s)\n  RAM: 20480 MB\n  Disk: 40960 MB\n'
                        'Production shared pool resources consumed:\n'
-                       '  CPU: 1\n  RAM: 0.500000 GB\n  Disk: 1.000000 GB\n'
+                       '  CPU: 1 core(s)\n  RAM: 512 MB\n  Disk: 1024 MB\n'
                        'Production dedicated pool resources consumed:\n'
-                       '  CPU: 2\n  RAM: 1.000000 GB\n  Disk: 2.000000 GB\n'
+                       '  CPU: 2 core(s)\n  RAM: 1024 MB\n  Disk: 2048 MB\n'
                        'Non-production shared pool resources consumed:\n'
-                       '  CPU: 3\n  RAM: 2.000000 GB\n  Disk: 4.000000 GB\n'
+                       '  CPU: 3 core(s)\n  RAM: 2048 MB\n  Disk: 4096 MB\n'
                        'Non-production dedicated pool resources consumed:\n'
-                       '  CPU: 4\n  RAM: 4.000000 GB\n  Disk: 8.000000 GB')
+                       '  CPU: 4 core(s)\n  RAM: 4096 MB\n  Disk: 8192 MB')
     assert expected_output == self._get_quota(True, ['quota', 'get', 'west/bozo'])
 
   def test_get_quota_with_no_consumption_json(self):

http://git-wip-us.apache.org/repos/asf/aurora/blob/d521dcd7/src/test/python/apache/aurora/client/cli/test_status.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/test_status.py b/src/test/python/apache/aurora/client/cli/test_status.py
index 7f36b46..14ef360 100644
--- a/src/test/python/apache/aurora/client/cli/test_status.py
+++ b/src/test/python/apache/aurora/client/cli/test_status.py
@@ -236,15 +236,15 @@ class TestJobStatus(AuroraClientCommandTest):
       expected = textwrap.dedent("""\
           Active tasks (3):
           \tTask role: bozo, env: test, name: woops, instance: 1, status: RUNNING on slavehost
-          \t  cpus: 2, ram: 2 MB, disk: 2 MB
+          \t  CPU: 2 core(s), RAM: 2 MB, Disk: 2 MB
           \t  events:
           \t   1970-11-23 ##:##:## RUNNING: Hi there
           \tTask role: bozo, env: test, name: woops, instance: 2, status: RUNNING on slavehost
-          \t  cpus: 2, ram: 2 MB, disk: 2 MB
+          \t  CPU: 2 core(s), RAM: 2 MB, Disk: 2 MB
           \t  events:
           \t   1970-11-23 ##:##:## RUNNING: Hi there
           \tTask role: bozo, env: test, name: woops, instance: 3, status: RUNNING on slavehost
-          \t  cpus: 2, ram: 2 MB, disk: 2 MB
+          \t  CPU: 2 core(s), RAM: 2 MB, Disk: 2 MB
           \t  events:
           \t   1970-11-23 ##:##:## RUNNING: Hi there
           Inactive tasks (0):
@@ -264,32 +264,32 @@ class TestJobStatus(AuroraClientCommandTest):
       expected = textwrap.dedent("""\
           Active tasks (3):
           \tTask role: bozo, env: test, name: woops, instance: 1, status: RUNNING on slavehost
-          \t  cpus: 2, ram: 2 MB, disk: 2 MB
+          \t  CPU: 2 core(s), RAM: 2 MB, Disk: 2 MB
           \t  events:
           \t   1970-11-23 ##:##:## RUNNING: Hi there
           \tTask role: bozo, env: test, name: woops, instance: 2, status: RUNNING on slavehost
-          \t  cpus: 2, ram: 2 MB, disk: 2 MB
+          \t  CPU: 2 core(s), RAM: 2 MB, Disk: 2 MB
           \t  events:
           \t   1970-11-23 ##:##:## RUNNING: Hi there
           \tTask role: bozo, env: test, name: woops, instance: 3, status: RUNNING on slavehost
-          \t  cpus: 2, ram: 2 MB, disk: 2 MB
+          \t  CPU: 2 core(s), RAM: 2 MB, Disk: 2 MB
           \t  events:
           \t   1970-11-23 ##:##:## RUNNING: Hi there
           Inactive tasks (3):
           \tTask role: bozo, env: test, name: woops, instance: 0, status: KILLED on slavehost
-          \t  cpus: 2, ram: 2 MB, disk: 2 MB
+          \t  CPU: 2 core(s), RAM: 2 MB, Disk: 2 MB
           \t  events:
           \t   1970-11-23 ##:##:## KILLED: Hi there
           \t   1970-11-23 ##:##:## FINISHED: Hi there
           \t   1970-11-23 ##:##:## FAILED: Hi there
           \tTask role: bozo, env: test, name: woops, instance: 1, status: FINISHED on slavehost
-          \t  cpus: 2, ram: 2 MB, disk: 2 MB
+          \t  CPU: 2 core(s), RAM: 2 MB, Disk: 2 MB
           \t  events:
           \t   1970-11-23 ##:##:## KILLED: Hi there
           \t   1970-11-23 ##:##:## FINISHED: Hi there
           \t   1970-11-23 ##:##:## FAILED: Hi there
           \tTask role: bozo, env: test, name: woops, instance: 2, status: FAILED on slavehost
-          \t  cpus: 2, ram: 2 MB, disk: 2 MB
+          \t  CPU: 2 core(s), RAM: 2 MB, Disk: 2 MB
           \t  events:
           \t   1970-11-23 ##:##:## KILLED: Hi there
           \t   1970-11-23 ##:##:## FINISHED: Hi there
@@ -310,21 +310,21 @@ class TestJobStatus(AuroraClientCommandTest):
       expected = textwrap.dedent("""\
           Active tasks (3):
           \tTask role: bozo, env: test, name: woops, instance: 1, status: RUNNING on slavehost
-          \t  cpus: 2, ram: 2 MB, disk: 2 MB
+          \t  CPU: 2 core(s), RAM: 2 MB, Disk: 2 MB
           \t  events:
           \t   1970-11-23 ##:##:## RUNNING: Hi there
           \t  metadata:
           \t\t  (key: 'meta', value: 'data')
           \t\t  (key: 'data', value: 'meta')
           \tTask role: bozo, env: test, name: woops, instance: 2, status: RUNNING on slavehost
-          \t  cpus: 2, ram: 2 MB, disk: 2 MB
+          \t  CPU: 2 core(s), RAM: 2 MB, Disk: 2 MB
           \t  events:
           \t   1970-11-23 ##:##:## RUNNING: Hi there
           \t  metadata:
           \t\t  (key: 'meta', value: 'data')
           \t\t  (key: 'data', value: 'meta')
           \tTask role: bozo, env: test, name: woops, instance: 3, status: RUNNING on slavehost
-          \t  cpus: 2, ram: 2 MB, disk: 2 MB
+          \t  CPU: 2 core(s), RAM: 2 MB, Disk: 2 MB
           \t  events:
           \t   1970-11-23 ##:##:## RUNNING: Hi there
           \t  metadata:

http://git-wip-us.apache.org/repos/asf/aurora/blob/d521dcd7/src/test/python/apache/aurora/config/test_resources.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/config/test_resources.py b/src/test/python/apache/aurora/config/test_resources.py
new file mode 100644
index 0000000..25a20f0
--- /dev/null
+++ b/src/test/python/apache/aurora/config/test_resources.py
@@ -0,0 +1,60 @@
+#
+# 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.
+#
+import unittest
+
+from apache.aurora.config.resource import ResourceDetails, ResourceManager, ResourceType
+
+from gen.apache.aurora.api.ttypes import Resource, ResourceAggregate, TaskConfig
+
+
+class TestResourceType(unittest.TestCase):
+  def test_from_resource(self):
+    assert ResourceType.from_resource(Resource(numCpus=1.0)) is ResourceType.CPUS
+    assert ResourceType.from_resource(Resource(ramMb=1)) is ResourceType.RAM_MB
+    assert ResourceType.from_resource(Resource(diskMb=0)) is ResourceType.DISK_MB
+    assert ResourceType.from_resource(Resource(namedPort='http')) is ResourceType.PORTS
+
+  def test_resource_value(self):
+    assert ResourceType.CPUS.resource_value(Resource(numCpus=1.0)) == 1.0
+
+
+class TestResourceManager(unittest.TestCase):
+  def test_resource_details(self):
+    details = ResourceManager.resource_details([Resource(ramMb=2), Resource(numCpus=1.0)])
+    assert len(details) == 2
+    assert details[0] == ResourceDetails(ResourceType.CPUS, 1.0)
+    assert details[1] == ResourceDetails(ResourceType.RAM_MB, 2)
+
+  def test_quantity_of(self):
+    quantity = ResourceManager.quantity_of(
+        ResourceManager.resource_details([Resource(ramMb=2), Resource(numCpus=1.0)]),
+        ResourceType.CPUS)
+    assert quantity == 1.0
+
+  def test_backfill_quota(self):
+    quota = ResourceAggregate(numCpus=1.0, ramMb=2, diskMb=3)
+    assert ResourceManager.resource_details_from_quota(quota) == [
+        ResourceDetails(ResourceType.CPUS, 1.0),
+        ResourceDetails(ResourceType.RAM_MB, 2),
+        ResourceDetails(ResourceType.DISK_MB, 3)
+    ]
+
+  def test_backfill_task(self):
+    task = TaskConfig(numCpus=1.0, ramMb=2, diskMb=3, requestedPorts=frozenset(['http']))
+    assert ResourceManager.resource_details_from_quota(task) == [
+        ResourceDetails(ResourceType.CPUS, 1.0),
+        ResourceDetails(ResourceType.RAM_MB, 2),
+        ResourceDetails(ResourceType.DISK_MB, 3),
+        ResourceDetails(ResourceType.PORTS, 'http'),
+    ]