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 2014/04/02 21:22:03 UTC

git commit: Filtering safe domain results by hosts.

Repository: incubator-aurora
Updated Branches:
  refs/heads/master 6cdc53806 -> 9906d2170


Filtering safe domain results by hosts.

Bugs closed: AURORA-295

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


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

Branch: refs/heads/master
Commit: 9906d217093568ed4c9cfe620862818f15ce4150
Parents: 6cdc538
Author: Maxim Khutornenko <ma...@apache.org>
Authored: Wed Apr 2 12:21:37 2014 -0700
Committer: Maxim Khutornenko <ma...@apache.org>
Committed: Wed Apr 2 12:21:37 2014 -0700

----------------------------------------------------------------------
 src/main/python/apache/aurora/client/api/sla.py | 21 ++++----
 .../apache/aurora/client/commands/admin.py      |  2 +-
 .../python/apache/aurora/client/api/test_sla.py | 50 ++++++++++++--------
 .../aurora/client/commands/test_admin_sla.py    |  8 ++--
 4 files changed, 47 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/9906d217/src/main/python/apache/aurora/client/api/sla.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/api/sla.py b/src/main/python/apache/aurora/client/api/sla.py
index 45153c7..c381de4 100644
--- a/src/main/python/apache/aurora/client/api/sla.py
+++ b/src/main/python/apache/aurora/client/api/sla.py
@@ -155,11 +155,12 @@ class DomainUpTimeSlaVector(object):
   JobUpTimeDetails = namedtuple('JobUpTimeDetails',
       ['job', 'predicted_percentage', 'safe', 'safe_in_secs'])
 
-  def __init__(self, cluster, tasks):
+  def __init__(self, cluster, tasks, hosts=None):
     self._cluster = cluster
     self._tasks = tasks
     self._now = time.time()
-    self._jobs, self._hosts = self._init_mappings()
+    self._tasks_by_job, self._jobs_by_host = self._init_mappings()
+    self._host_filter = hosts
 
   def get_safe_hosts(self, percentage, duration, job_limits=None):
     """Returns hosts safe to restart with respect to their job SLA.
@@ -172,7 +173,10 @@ class DomainUpTimeSlaVector(object):
                      replaces default percentage/duration within the job context.
     """
     safe_hosts = defaultdict(list)
-    for host, job_keys in self._hosts.items():
+    for host, job_keys in self._jobs_by_host.items():
+      if self._host_filter and host not in self._host_filter:
+        continue
+
       safe_limits = []
       for job_key in job_keys:
         job_duration = duration
@@ -192,7 +196,7 @@ class DomainUpTimeSlaVector(object):
 
     return safe_hosts
 
-  def probe_hosts(self, percentage, duration, hosts):
+  def probe_hosts(self, percentage, duration):
     """Returns predicted job SLAs following the removal of provided hosts.
 
        For every given host creates a list of JobUpTimeDetails with predicted job SLA details
@@ -203,11 +207,10 @@ class DomainUpTimeSlaVector(object):
        Arguments:
        percentage -- task up count percentage.
        duration -- task uptime duration in seconds.
-       hosts -- list of hosts to probe for job SLA changes.
     """
     probed_hosts = defaultdict(list)
-    for host in hosts:
-      for job_key in self._hosts.get(host, []):
+    for host in self._host_filter or []:
+      for job_key in self._jobs_by_host.get(host, []):
         filtered_percentage, total_count, filtered_vector = self._simulate_host_down(
             job_key, host, duration)
 
@@ -225,7 +228,7 @@ class DomainUpTimeSlaVector(object):
     return probed_hosts
 
   def _simulate_host_down(self, job_key, host, duration):
-    unfiltered_tasks = self._jobs[job_key]
+    unfiltered_tasks = self._tasks_by_job[job_key]
 
     # Get total job task count to use in SLA calculation.
     total_count = len(unfiltered_tasks)
@@ -276,7 +279,7 @@ class Sla(object):
     """
     tasks = self._get_tasks(task_query(hosts=hosts)) if hosts else None
     job_keys = set(job_key_from_scheduled(t, cluster) for t in tasks) if tasks else None
-    return DomainUpTimeSlaVector(cluster, self._get_tasks(task_query(job_keys=job_keys)))
+    return DomainUpTimeSlaVector(cluster, self._get_tasks(task_query(job_keys=job_keys)), hosts)
 
   def _get_tasks(self, task_query):
     resp = self._scheduler.getTasksStatus(task_query)

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/9906d217/src/main/python/apache/aurora/client/commands/admin.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/commands/admin.py b/src/main/python/apache/aurora/client/commands/admin.py
index f411b79..2b37101 100644
--- a/src/main/python/apache/aurora/client/commands/admin.py
+++ b/src/main/python/apache/aurora/client/commands/admin.py
@@ -441,7 +441,7 @@ def sla_probe_hosts(cluster, percentage, duration):
   hosts = parse_hosts(options.filename, options.hosts)
 
   vector = AuroraClientAPI(CLUSTERS[cluster], options.verbosity).sla_get_safe_domain_vector(hosts)
-  probed_hosts = vector.probe_hosts(sla_percentage, sla_duration.as_(Time.SECONDS), hosts)
+  probed_hosts = vector.probe_hosts(sla_percentage, sla_duration.as_(Time.SECONDS))
 
   results = []
   for host, job_details in sorted(probed_hosts.items()):

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/9906d217/src/test/python/apache/aurora/client/api/test_sla.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/api/test_sla.py b/src/test/python/apache/aurora/client/api/test_sla.py
index ff11b1f..dedc8f1 100644
--- a/src/test/python/apache/aurora/client/api/test_sla.py
+++ b/src/test/python/apache/aurora/client/api/test_sla.py
@@ -126,26 +126,28 @@ class SlaTest(unittest.TestCase):
       )
     self.expect_task_status_call_cluster_scoped()
 
-  def assert_probe_hosts_result(self, host, percent, duration, f_percent, safe=True, wait_time=0):
-    vector = self._sla.get_domain_uptime_vector(self._cluster)
-    result = vector.probe_hosts(percent, duration, [host])
-    assert 1 == len(result), ('Expected length:%s Actual length:%s' % (1, len(result)))
+  def assert_probe_hosts_result(self, hosts, percent, duration):
+    vector = self._sla.get_domain_uptime_vector(self._cluster, hosts)
+    result = vector.probe_hosts(percent, duration)
+    assert len(hosts) == len(result), ('Expected length:%s Actual length:%s' % (1, len(result)))
+    return result
+
+  def assert_probe_host_job_details(self, result, host, f_percent, safe=True, wait_time=0):
     assert host in result, ('Expected host:%s not found in result' % host)
 
     job_details = result[host][0]
     assert job_details.job.name == self._name, (
-        'Expected job:%s Actual:%s' % (self._name, job_details.job.name)
+      'Expected job:%s Actual:%s' % (self._name, job_details.job.name)
     )
     assert job_details.predicted_percentage == f_percent, (
-        'Expected percentage:%s Actual:%s' % (f_percent, job_details.predicted_percentage)
+      'Expected percentage:%s Actual:%s' % (f_percent, job_details.predicted_percentage)
     )
     assert job_details.safe == safe, (
-        'Expected safe:%s Actual:%s' % (safe, job_details.safe)
+      'Expected safe:%s Actual:%s' % (safe, job_details.safe)
     )
     assert job_details.safe_in_secs == wait_time, (
       'Expected safe:%s Actual:%s' % (wait_time, job_details.safe_in_secs)
     )
-    self.expect_task_status_call_cluster_scoped()
 
   def expect_task_status_call_job_scoped(self):
     self._scheduler.getTasksStatus.assert_called_once_with(
@@ -157,7 +159,7 @@ class SlaTest(unittest.TestCase):
     )
 
   def expect_task_status_call_cluster_scoped(self):
-    self._scheduler.getTasksStatus.assert_called_once_with(TaskQuery(statuses=LIVE_STATES))
+    self._scheduler.getTasksStatus.assert_called_with(TaskQuery(statuses=LIVE_STATES))
 
 
   def test_count_0(self):
@@ -287,27 +289,32 @@ class SlaTest(unittest.TestCase):
     self.expect_task_status_call_cluster_scoped()
 
 
-  def test_probe_hosts_no_tasks(self):
+  def test_probe_hosts_no_hosts(self):
     self.mock_get_tasks([])
     vector = self._sla.get_domain_uptime_vector(self._cluster)
-    assert 0 == len(vector.probe_hosts(90, 200, ['h1', 'h2']))
-    self.expect_task_status_call_cluster_scoped()
+    assert 0 == len(vector.probe_hosts(90, 200))
+
+  def test_probe_hosts_no_tasks(self):
+    self.mock_get_tasks([])
+    vector = self._sla.get_domain_uptime_vector(self._cluster, hosts=['h1', 'h2'])
+    assert 0 == len(vector.probe_hosts(90, 200))
 
   def test_probe_hosts_no_result(self):
     self.mock_get_tasks([
         self.create_task(100, 1, 'h3', 'j1'),
         self.create_task(100, 1, 'h4', 'j2')
     ])
-    vector = self._sla.get_domain_uptime_vector(self._cluster)
-    assert 0 == len(vector.probe_hosts(90, 200, ['h1', 'h2']))
-    self.expect_task_status_call_cluster_scoped()
+    vector = self._sla.get_domain_uptime_vector(self._cluster, ['h1', 'h2'])
+    assert 0 == len(vector.probe_hosts(90, 200))
 
   def test_probe_hosts_safe(self):
     self.mock_get_tasks([
         self.create_task(100, 1, 'h1', self._name),
         self.create_task(100, 2, 'h2', self._name),
     ])
-    self.assert_probe_hosts_result('h1', 20, 100, 50.0)
+    result = self.assert_probe_hosts_result(['h1', 'h2'], 20, 100)
+    self.assert_probe_host_job_details(result, 'h1', 50.0)
+    self.assert_probe_host_job_details(result, 'h2', 50.0)
 
   def test_probe_hosts_not_safe(self):
     self.mock_get_tasks([
@@ -316,7 +323,8 @@ class SlaTest(unittest.TestCase):
         self.create_task(300, 3, 'h3', self._name),
         self.create_task(400, 4, 'h4', self._name),
     ])
-    self.assert_probe_hosts_result('h1', 75, 300, 50.0, False, 100)
+    result = self.assert_probe_hosts_result(['h1', 'h2', 'h3', 'h4'], 75, 300)
+    self.assert_probe_host_job_details(result, 'h1', 50.0, False, 100)
 
   def test_probe_hosts_not_safe_infeasible(self):
     self.mock_get_tasks([
@@ -325,7 +333,8 @@ class SlaTest(unittest.TestCase):
         self.create_task(300, 3, 'h3', self._name),
         self.create_task(400, 4, 'h4', self._name),
     ])
-    self.assert_probe_hosts_result('h1', 80, 300, 50.0, False, None)
+    result = self.assert_probe_hosts_result(['h1', 'h2', 'h3', 'h4'], 80, 300)
+    self.assert_probe_host_job_details(result, 'h1', 50.0, False, None)
 
   def test_probe_hosts_non_prod_ignored(self):
     self.mock_get_tasks([
@@ -334,9 +343,8 @@ class SlaTest(unittest.TestCase):
         self.create_task(300, 3, 'h3', self._name, False),
         self.create_task(400, 4, 'h4', self._name, False),
     ])
-    vector = self._sla.get_domain_uptime_vector(self._cluster)
-    assert 0 == len(vector.probe_hosts(50, 80, ['h1']))
-    self.expect_task_status_call_cluster_scoped()
+    vector = self._sla.get_domain_uptime_vector(self._cluster, ['h1', 'h2'])
+    assert 0 == len(vector.probe_hosts(90, 200))
 
 
   def test_get_domain_uptime_vector_with_hosts(self):

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/9906d217/src/test/python/apache/aurora/client/commands/test_admin_sla.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/commands/test_admin_sla.py b/src/test/python/apache/aurora/client/commands/test_admin_sla.py
index 2f09cb1..aaf6c7f 100644
--- a/src/test/python/apache/aurora/client/commands/test_admin_sla.py
+++ b/src/test/python/apache/aurora/client/commands/test_admin_sla.py
@@ -34,7 +34,9 @@ class TestAdminSlaListSafeDomainCommand(AuroraClientCommandTest):
   @classmethod
   def setup_mock_options(cls, exclude=None, include=None, override=None,
                          exclude_list=None, include_list=None, list_jobs=False):
-    mock_options = Mock()
+    mock_options = Mock(spec=['exclude_filename', 'exclude_hosts', 'include_filename',
+        'include_hosts', 'override_filename', 'list_jobs', 'verbosity', 'disable_all_hooks'])
+
     mock_options.exclude_filename = exclude
     mock_options.exclude_hosts = exclude_list
     mock_options.include_filename = include
@@ -315,7 +317,7 @@ class TestAdminSlaProbeHostsCommand(AuroraClientCommandTest):
       sla_probe_hosts(['west', '90', '200s'])
 
       mock_api.return_value.sla_get_safe_domain_vector.assert_called_once_with(hosts)
-      mock_vector.probe_hosts.assert_called_once_with(90.0, 200.0, hosts)
+      mock_vector.probe_hosts.assert_called_once_with(90.0, 200.0)
       mock_print_results.assert_called_once_with([
           'h0\twest/role/env/job0\t80.00\tTrue\t0',
           'h1\twest/role/env/job1\t80.00\tTrue\t0'
@@ -343,7 +345,7 @@ class TestAdminSlaProbeHostsCommand(AuroraClientCommandTest):
         sla_probe_hosts(['west', '90', '200s'])
 
         mock_api.return_value.sla_get_safe_domain_vector.assert_called_once_with(['h0'])
-        mock_vector.probe_hosts.assert_called_once_with(90.0, 200.0, ['h0'])
+        mock_vector.probe_hosts.assert_called_once_with(90.0, 200.0)
         mock_print_results.assert_called_once_with([
             'h0\twest/role/env/job0\t80.00\tFalse\tn/a'
         ])