You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by wf...@apache.org on 2017/11/28 01:01:07 UTC

aurora git commit: Remove task level resource fields from thrift interface

Repository: aurora
Updated Branches:
  refs/heads/master 0f3dc939e -> 21af250c9


Remove task level resource fields from thrift interface

Bugs closed: AURORA-1707

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


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

Branch: refs/heads/master
Commit: 21af250c9e781d7d4a121c74f561cbda36245040
Parents: 0f3dc93
Author: Nicolás Donatucci <nd...@medallia.com>
Authored: Mon Nov 27 17:01:03 2017 -0800
Committer: Bill Farner <wf...@apache.org>
Committed: Mon Nov 27 17:01:03 2017 -0800

----------------------------------------------------------------------
 RELEASE-NOTES.md                                |  1 +
 .../thrift/org/apache/aurora/gen/api.thrift     |  8 ---
 .../aurora/scheduler/base/TaskTestUtil.java     |  4 --
 .../apache/aurora/client/cli/diff_formatter.py  |  5 +-
 .../python/apache/aurora/client/cli/jobs.py     |  6 +-
 .../apache/aurora/scheduler/http/TestUtils.java | 13 ++++-
 .../aurora/scheduler/thrift/Fixtures.java       |  4 --
 .../updater/UpdateAgentReserverImplTest.java    |  8 ++-
 .../apache/aurora/client/cli/test_status.py     | 59 +++++++++++++-------
 .../python/apache/aurora/client/cli/util.py     |  8 ++-
 .../apache/aurora/config/test_resources.py      | 11 +---
 11 files changed, 64 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/21af250c/RELEASE-NOTES.md
----------------------------------------------------------------------
diff --git a/RELEASE-NOTES.md b/RELEASE-NOTES.md
index e622a8d..2d3c423 100644
--- a/RELEASE-NOTES.md
+++ b/RELEASE-NOTES.md
@@ -17,6 +17,7 @@
 
 - Removed the ability to recover from SQL-based backups and snapshots.  An 0.20.0 scheduler
   will not be able to recover backups or replicated log data created prior to 0.19.0.
+- Removed task level resource fields (`numCpus`, `ramMb`, `diskMb`, `requestedPorts`).
 
 0.19.0
 ======

http://git-wip-us.apache.org/repos/asf/aurora/blob/21af250c/api/src/main/thrift/org/apache/aurora/gen/api.thrift
----------------------------------------------------------------------
diff --git a/api/src/main/thrift/org/apache/aurora/gen/api.thrift b/api/src/main/thrift/org/apache/aurora/gen/api.thrift
index 2978f6d..c9f4210 100644
--- a/api/src/main/thrift/org/apache/aurora/gen/api.thrift
+++ b/api/src/main/thrift/org/apache/aurora/gen/api.thrift
@@ -232,12 +232,6 @@ struct TaskConfig {
  /** contains the role component of JobKey */
  17: Identity owner
   7: bool isService
-  // TODO(maxim): Deprecated. See AURORA-1707.
-  8: double numCpus
-  // TODO(maxim): Deprecated. See AURORA-1707.
-  9: i64 ramMb
-  // TODO(maxim): Deprecated. See AURORA-1707.
- 10: i64 diskMb
  11: i32 priority
  13: i32 maxTaskFailures
  // TODO(mnurolahzade): Deprecated. See AURORA-1708.
@@ -249,8 +243,6 @@ struct TaskConfig {
  32: set<Resource> resources
 
  20: set<Constraint> constraints
- /** a list of named ports this task requests */
- 21: set<string> requestedPorts
  /** Resources to retrieve with Mesos Fetcher */
  33: optional set<MesosFetcherURI> mesosFetcherUris
  /**

http://git-wip-us.apache.org/repos/asf/aurora/blob/21af250c/src/main/java/org/apache/aurora/scheduler/base/TaskTestUtil.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/base/TaskTestUtil.java b/src/main/java/org/apache/aurora/scheduler/base/TaskTestUtil.java
index dd64a5b..5fe7b9b 100644
--- a/src/main/java/org/apache/aurora/scheduler/base/TaskTestUtil.java
+++ b/src/main/java/org/apache/aurora/scheduler/base/TaskTestUtil.java
@@ -116,9 +116,6 @@ public final class TaskTestUtil {
         .setJob(job.newBuilder())
         .setOwner(new Identity().setUser(job.getRole() + "-user"))
         .setIsService(true)
-        .setNumCpus(1.0)
-        .setRamMb(1024)
-        .setDiskMb(1024)
         .setPriority(1)
         .setMaxTaskFailures(-1)
         .setProduction(true)
@@ -132,7 +129,6 @@ public final class TaskTestUtil {
             new Constraint(
                 "limitConstraint",
                 TaskConstraint.limit(new LimitConstraint(5)))))
-        .setRequestedPorts(ImmutableSet.of("http"))
         .setTaskLinks(ImmutableMap.of("http", "link", "admin", "otherLink"))
         .setContactEmail("foo@bar.com")
         .setMetadata(ImmutableSet.of(new Metadata("key", "value")))

http://git-wip-us.apache.org/repos/asf/aurora/blob/21af250c/src/main/python/apache/aurora/client/cli/diff_formatter.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/diff_formatter.py b/src/main/python/apache/aurora/client/cli/diff_formatter.py
index 7871777..2f66e29 100644
--- a/src/main/python/apache/aurora/client/cli/diff_formatter.py
+++ b/src/main/python/apache/aurora/client/cli/diff_formatter.py
@@ -63,9 +63,6 @@ class DiffFormatter(object):
     if task.resources:
       task.resources = sorted(task.resources, key=str)
 
-    if task.requestedPorts:
-      task.requestedPorts = sorted(task.requestedPorts, key=str)
-
     if task.mesosFetcherUris:
       task.mesosFetcherUris = sorted(task.mesosFetcherUris, key=str)
 
@@ -129,6 +126,8 @@ class DiffFormatter(object):
           format_ranges(r for r in chain.from_iterable(s.instances for s in summaries)))
 
   def diff_no_update_details(self, local_tasks):
+    # Deepcopy is important here as tasks will be modified for printing.
+    local_tasks = [deepcopy(t) for t in local_tasks]
     api = self.context.get_api(self.cluster)
     resp = api.query(api.build_query(self.role, self.name, env=self.env, statuses=ACTIVE_STATES))
     self.context.log_response_and_raise(

http://git-wip-us.apache.org/repos/asf/aurora/blob/21af250c/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 cbae387..536d04a 100644
--- a/src/main/python/apache/aurora/client/cli/jobs.py
+++ b/src/main/python/apache/aurora/client/cli/jobs.py
@@ -20,7 +20,6 @@ import pprint
 import textwrap
 import webbrowser
 from collections import namedtuple
-from copy import deepcopy
 from datetime import datetime
 
 from thrift.protocol import TJSONProtocol
@@ -201,10 +200,7 @@ class DiffCommand(Verb):
         err_code=EXIT_INVALID_CONFIGURATION,
         err_msg="Error loading configuration")
     local_task = resp.result.populateJobResult.taskConfig
-    # Deepcopy is important here as tasks will be modified for printing.
-    local_tasks = [
-        deepcopy(local_task) for _ in range(config.instances())
-    ]
+    local_tasks = [local_task for _ in range(config.instances())]
     instances = (None if context.options.instance_spec.instance == ALL_INSTANCES else
                  context.options.instance_spec.instance)
     formatter = DiffFormatter(context, config, cluster, role, env, name)

http://git-wip-us.apache.org/repos/asf/aurora/blob/21af250c/src/test/java/org/apache/aurora/scheduler/http/TestUtils.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/http/TestUtils.java b/src/test/java/org/apache/aurora/scheduler/http/TestUtils.java
index 689482c..eb6b39f 100644
--- a/src/test/java/org/apache/aurora/scheduler/http/TestUtils.java
+++ b/src/test/java/org/apache/aurora/scheduler/http/TestUtils.java
@@ -15,6 +15,8 @@ package org.apache.aurora.scheduler.http;
 
 import com.google.common.annotations.VisibleForTesting;
 
+import com.google.common.collect.ImmutableSet;
+
 import org.apache.aurora.gen.AssignedTask;
 import org.apache.aurora.gen.ScheduleStatus;
 import org.apache.aurora.gen.ScheduledTask;
@@ -22,6 +24,10 @@ import org.apache.aurora.gen.TaskConfig;
 import org.apache.aurora.scheduler.storage.entities.IJobKey;
 import org.apache.aurora.scheduler.storage.entities.IScheduledTask;
 
+import static org.apache.aurora.gen.Resource.diskMb;
+import static org.apache.aurora.gen.Resource.numCpus;
+import static org.apache.aurora.gen.Resource.ramMb;
+
 public final class TestUtils {
 
   private TestUtils() { }
@@ -49,8 +55,9 @@ public final class TestUtils {
             .setTaskId(id)
             .setTask(new TaskConfig()
                 .setJob(jobKey.newBuilder())
-                .setNumCpus(numCPUs)
-                .setRamMb(ramMB)
-                .setDiskMb(diskMB))));
+                    .setResources(ImmutableSet.of(
+                        numCpus(numCPUs),
+                        ramMb(ramMB),
+                        diskMb(diskMB))))));
   }
 }

http://git-wip-us.apache.org/repos/asf/aurora/blob/21af250c/src/test/java/org/apache/aurora/scheduler/thrift/Fixtures.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/thrift/Fixtures.java b/src/test/java/org/apache/aurora/scheduler/thrift/Fixtures.java
index d1b5e40..e1b8eaa 100644
--- a/src/test/java/org/apache/aurora/scheduler/thrift/Fixtures.java
+++ b/src/test/java/org/apache/aurora/scheduler/thrift/Fixtures.java
@@ -102,12 +102,8 @@ final class Fixtures {
         .setOwner(IDENTITY)
         .setContactEmail("testing@twitter.com")
         .setExecutorConfig(new ExecutorConfig(apiConstants.AURORA_EXECUTOR_NAME, "data"))
-        .setNumCpus(1)
-        .setRamMb(1024)
-        .setDiskMb(1024)
         .setProduction(production)
         .setTier(production ? TaskTestUtil.PROD_TIER_NAME : TaskTestUtil.DEV_TIER_NAME)
-        .setRequestedPorts(ImmutableSet.of())
         .setTaskLinks(ImmutableMap.of())
         .setMaxTaskFailures(1)
         .setConstraints(ImmutableSet.of())

http://git-wip-us.apache.org/repos/asf/aurora/blob/21af250c/src/test/java/org/apache/aurora/scheduler/updater/UpdateAgentReserverImplTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/updater/UpdateAgentReserverImplTest.java b/src/test/java/org/apache/aurora/scheduler/updater/UpdateAgentReserverImplTest.java
index 1bc2a77..7f17be0 100644
--- a/src/test/java/org/apache/aurora/scheduler/updater/UpdateAgentReserverImplTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/updater/UpdateAgentReserverImplTest.java
@@ -17,6 +17,7 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 
 import org.apache.aurora.common.testing.easymock.EasyMockTest;
+import org.apache.aurora.gen.Resource;
 import org.apache.aurora.gen.TaskConfig;
 import org.apache.aurora.scheduler.base.InstanceKeys;
 import org.apache.aurora.scheduler.base.JobKeys;
@@ -46,9 +47,10 @@ public class UpdateAgentReserverImplTest extends EasyMockTest {
     return TaskGroupKey.from(ITaskConfig.build(
         new TaskConfig()
             .setJob(key.getJobKey().newBuilder())
-            .setNumCpus(1.0)
-            .setRamMb(1L)
-            .setDiskMb(1L)));
+            .setResources(ImmutableSet.of(
+                Resource.numCpus(1.0),
+                Resource.ramMb(1L),
+                Resource.diskMb(1L)))));
   }
 
   @Before

http://git-wip-us.apache.org/repos/asf/aurora/blob/21af250c/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 b0b7f96..d462dc0 100644
--- a/src/test/python/apache/aurora/client/cli/test_status.py
+++ b/src/test/python/apache/aurora/client/cli/test_status.py
@@ -31,6 +31,7 @@ from gen.apache.aurora.api.ttypes import (
     JobConfiguration,
     JobKey,
     Metadata,
+    Resource,
     ResponseCode,
     Result,
     ScheduledTask,
@@ -148,13 +149,13 @@ class TestJobStatus(AuroraClientCommandTest):
           task=TaskConfig(
             job=JobKey(role="nobody", environment="prod", name='flibber'),
             isService=False,
-            numCpus=2,
-            ramMb=2048,
-            diskMb=4096,
+            resources=frozenset(
+              [Resource(numCpus=2),
+               Resource(ramMb=2048),
+               Resource(diskMb=4096)]),
             priority=7,
             maxTaskFailures=3,
-            production=False,
-            requestedPorts=["http"]),
+            production=False),
           assignedPorts={"http": 1001},
           instanceId=instance),
         status=2,
@@ -426,6 +427,7 @@ class TestJobStatus(AuroraClientCommandTest):
       cmd = AuroraCommandLine()
       cmd.execute(['job', 'status', '--write-json', 'west/bozo/test/hello'])
       actual = re.sub("\\d\\d:\\d\\d:\\d\\d", "##:##:##", '\n'.join(mock_context.get_out()))
+      actual_sorted = json.loads(actual)
       expected = [
           {
             "active": [
@@ -437,9 +439,7 @@ class TestJobStatus(AuroraClientCommandTest):
                     "container": {
                       "mesos": {}
                     },
-                    "requestedPorts": [
-                      "http"
-                    ],
+                    "maxTaskFailures": 3,
                     "priority": 7,
                     "job": {
                       "environment": "prod",
@@ -447,10 +447,17 @@ class TestJobStatus(AuroraClientCommandTest):
                       "name": "flibber"
                     },
                     "production": False,
-                    "diskMb": 4096,
-                    "ramMb": 2048,
-                    "maxTaskFailures": 3,
-                    "numCpus": 2
+                    "resources": [
+                      {
+                        "numCpus": 2
+                      },
+                      {
+                        "ramMb": 2048
+                      },
+                      {
+                        "diskMb": 4096
+                      }
+                    ]
                   },
                   "taskId": "task_0",
                   "instanceId": 0,
@@ -488,9 +495,7 @@ class TestJobStatus(AuroraClientCommandTest):
                     "container": {
                       "mesos": {}
                     },
-                    "requestedPorts": [
-                      "http"
-                    ],
+                    "maxTaskFailures": 3,
                     "priority": 7,
                     "job": {
                       "environment": "prod",
@@ -498,10 +503,17 @@ class TestJobStatus(AuroraClientCommandTest):
                       "name": "flibber"
                     },
                     "production": False,
-                    "diskMb": 4096,
-                    "ramMb": 2048,
-                    "maxTaskFailures": 3,
-                    "numCpus": 2
+                    "resources": [
+                      {
+                        "numCpus": 2
+                      },
+                      {
+                        "ramMb": 2048
+                      },
+                      {
+                        "diskMb": 4096
+                      }
+                    ]
                   },
                   "taskId": "task_1",
                   "instanceId": 1,
@@ -536,7 +548,14 @@ class TestJobStatus(AuroraClientCommandTest):
             "inactive": []
           }
       ]
-      assert json.loads(actual) == expected
+    for entry in actual_sorted[0]["active"]:
+      entry["assignedTask"]["task"]["resources"] = sorted(
+        entry["assignedTask"]["task"]["resources"], key=str)
+    for entry in expected[0]["active"]:
+      entry["assignedTask"]["task"]["resources"] = sorted(
+        entry["assignedTask"]["task"]["resources"], key=str)
+
+    assert actual_sorted == expected
 
   def test_status_job_not_found(self):
     """Regression test: there was a crasher bug when metadata was None."""

http://git-wip-us.apache.org/repos/asf/aurora/blob/21af250c/src/test/python/apache/aurora/client/cli/util.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/util.py b/src/test/python/apache/aurora/client/cli/util.py
index 476b8c2..8b3b4f5 100644
--- a/src/test/python/apache/aurora/client/cli/util.py
+++ b/src/test/python/apache/aurora/client/cli/util.py
@@ -32,6 +32,7 @@ from gen.apache.aurora.api.ttypes import (
     GetTierConfigResult,
     JobKey,
     JobUpdateSummary,
+    Resource,
     Response,
     ResponseCode,
     ResponseDetail,
@@ -227,9 +228,10 @@ class AuroraClientCommandTest(unittest.TestCase):
         executorConfig=ExecutorConfig(data='{"fake": "data"}'),
         metadata=[],
         job=JobKey(role=cls.TEST_ROLE, environment=cls.TEST_ENV, name=name),
-        numCpus=2,
-        ramMb=2,
-        diskMb=2)
+        resources=frozenset(
+            [Resource(numCpus=2),
+             Resource(ramMb=2),
+             Resource(diskMb=2)]))
 
   @classmethod
   def create_scheduled_tasks(cls):

http://git-wip-us.apache.org/repos/asf/aurora/blob/21af250c/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
index 25a20f0..f43bad7 100644
--- a/src/test/python/apache/aurora/config/test_resources.py
+++ b/src/test/python/apache/aurora/config/test_resources.py
@@ -15,7 +15,7 @@ import unittest
 
 from apache.aurora.config.resource import ResourceDetails, ResourceManager, ResourceType
 
-from gen.apache.aurora.api.ttypes import Resource, ResourceAggregate, TaskConfig
+from gen.apache.aurora.api.ttypes import Resource, ResourceAggregate
 
 
 class TestResourceType(unittest.TestCase):
@@ -49,12 +49,3 @@ class TestResourceManager(unittest.TestCase):
         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'),
-    ]