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

aurora git commit: Remove url related methods out of AuroraCommandContext

Repository: aurora
Updated Branches:
  refs/heads/master 5506bfa27 -> e8e1caaa7


Remove url related methods out of AuroraCommandContext

This is a refactor of AuroraCommandContext which removes all url related methods
out of it. The objective of this refactor is to remove functionality from
AuroraCommandContext to allow for easier testing of commands. This commit also
adds two tests for commands which were using the url related functionality but
lacked test coverage.

Testing Done:
./pants test.pytest --no-fast src/test/python/apache/aurora/client/cli::

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


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

Branch: refs/heads/master
Commit: e8e1caaa7cd7f6edc0d97cd0d42bcf4a0061c06f
Parents: 5506bfa
Author: Zameer Manji <zm...@apache.org>
Authored: Tue Apr 7 12:49:40 2015 -0700
Committer: zmanji@apache.org <zm...@twitter.com>
Committed: Tue Apr 7 12:49:40 2015 -0700

----------------------------------------------------------------------
 src/main/python/apache/aurora/client/base.py    | 13 +++++++++-
 .../python/apache/aurora/client/cli/context.py  | 25 +-------------------
 .../python/apache/aurora/client/cli/cron.py     |  6 +++--
 .../python/apache/aurora/client/cli/jobs.py     | 17 ++++++++-----
 .../python/apache/aurora/client/cli/update.py   |  7 ++++--
 .../apache/aurora/client/cli/test_create.py     |  6 +++--
 .../apache/aurora/client/cli/test_cron.py       |  9 ++++---
 .../apache/aurora/client/cli/test_kill.py       | 21 ++++++++++++++++
 .../apache/aurora/client/cli/test_open.py       | 16 ++++++++-----
 .../apache/aurora/client/cli/test_restart.py    | 23 +++++++++++++++++-
 .../python/apache/aurora/client/cli/util.py     | 11 +++++----
 11 files changed, 102 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/e8e1caaa/src/main/python/apache/aurora/client/base.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/base.py b/src/main/python/apache/aurora/client/base.py
index c72f2f7..df53b90 100644
--- a/src/main/python/apache/aurora/client/base.py
+++ b/src/main/python/apache/aurora/client/base.py
@@ -173,7 +173,7 @@ GROUPING_OPTION = optparse.Option(
         ', '.join(GROUPING_FUNCTIONS.keys())))
 
 
-def synthesize_url(scheduler_url, role=None, env=None, job=None):
+def synthesize_url(scheduler_url, role=None, env=None, job=None, update_id=None):
   if not scheduler_url:
     log.warning("Unable to find scheduler web UI!")
     return None
@@ -190,9 +190,20 @@ def synthesize_url(scheduler_url, role=None, env=None, job=None):
       scheduler_url += '/' + env
       if job:
         scheduler_url += '/' + job
+        if update_id:
+          scheduler_url += '/' + update_id
   return scheduler_url
 
 
+def get_job_page(api, jobkey):
+  return synthesize_url(api.scheduler_proxy.scheduler_client().url, jobkey.role,
+                        jobkey.env, jobkey.name)
+
+
+def get_update_page(api, jobkey, update_id):
+  return synthesize_url(api.scheduler_proxy.scheduler_client().url, jobkey.role,
+                        jobkey.env, jobkey.name, update_id)
+
 AURORA_V2_USER_AGENT_NAME = 'Aurora V2'
 AURORA_ADMIN_USER_AGENT_NAME = 'Aurora Admin'
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/e8e1caaa/src/main/python/apache/aurora/client/cli/context.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/context.py b/src/main/python/apache/aurora/client/cli/context.py
index e75c6cb..3d4a428 100644
--- a/src/main/python/apache/aurora/client/cli/context.py
+++ b/src/main/python/apache/aurora/client/cli/context.py
@@ -17,7 +17,7 @@ from __future__ import print_function
 import logging
 from fnmatch import fnmatch
 
-from apache.aurora.client.base import AURORA_V2_USER_AGENT_NAME, combine_messages, synthesize_url
+from apache.aurora.client.base import AURORA_V2_USER_AGENT_NAME, combine_messages
 from apache.aurora.client.cli import (
     Context,
     EXIT_API_ERROR,
@@ -84,29 +84,6 @@ class AuroraCommandContext(Context):
     except Exception as e:
       raise self.CommandError(EXIT_INVALID_CONFIGURATION, 'Error loading configuration: %s' % e)
 
-  def open_page(self, url):
-    import webbrowser
-    webbrowser.open_new_tab(url)
-
-  def open_job_page(self, api, jobkey):
-    """Opens the page for a job in the system web browser."""
-    self.open_page(self.get_job_page(api, jobkey))
-
-  def get_job_page(self, api, jobkey):
-    return synthesize_url(api.scheduler_proxy.scheduler_client().url, jobkey.role,
-        jobkey.env, jobkey.name)
-
-  def get_update_page(self, api, cluster, update_key):
-    return "%s/%s" % (
-        self.get_job_page(api, AuroraJobKey.from_thrift(cluster, update_key.job)),
-        update_key.id)
-
-  def open_scheduler_page(self, cluster, role, env, name):
-    """Open a scheduler page"""
-    api = self.get_api(cluster)
-    self.open_page(synthesize_url(api.scheduler_proxy.scheduler_client().url,
-        role, env, name))
-
   def log_response_and_raise(self, resp, err_code=EXIT_API_ERROR, err_msg="Command failure:"):
     if resp.responseCode == ResponseCode.OK:
       msg = combine_messages(resp)

http://git-wip-us.apache.org/repos/asf/aurora/blob/e8e1caaa/src/main/python/apache/aurora/client/cli/cron.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/cron.py b/src/main/python/apache/aurora/client/cli/cron.py
index 732135f..6376fd0 100644
--- a/src/main/python/apache/aurora/client/cli/cron.py
+++ b/src/main/python/apache/aurora/client/cli/cron.py
@@ -17,7 +17,9 @@
 from __future__ import print_function
 
 import textwrap
+import webbrowser
 
+from apache.aurora.client.base import get_job_page
 from apache.aurora.client.cli import (
     EXIT_COMMAND_FAILURE,
     EXIT_INVALID_PARAMETER,
@@ -63,7 +65,7 @@ class Schedule(Verb):
         err_msg=("Error scheduling cron job %s:" % context.options.jobspec))
 
     context.print_out("Cron job scheduled, status can be viewed at %s"
-        % context.get_job_page(api, context.options.jobspec))
+        % get_job_page(api, context.options.jobspec))
 
     return EXIT_OK
 
@@ -111,7 +113,7 @@ class Start(Verb):
     context.log_response_and_raise(resp,
         err_msg=("Error starting cron job %s:" % context.options.jobspec))
     if context.options.open_browser:
-      context.open_job_page(api, context.options.jobspec)
+      webbrowser.open_new_tab(get_job_page(api, context.options.jobspec))
     return EXIT_OK
 
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/e8e1caaa/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 2d82942..6d82a7e 100644
--- a/src/main/python/apache/aurora/client/cli/jobs.py
+++ b/src/main/python/apache/aurora/client/cli/jobs.py
@@ -21,6 +21,7 @@ import pprint
 import subprocess
 import textwrap
 import time
+import webbrowser
 from collections import namedtuple
 from copy import deepcopy
 from datetime import datetime
@@ -31,6 +32,7 @@ from thrift.TSerialization import serialize
 
 from apache.aurora.client.api.job_monitor import JobMonitor
 from apache.aurora.client.api.updater_util import UpdaterConfig
+from apache.aurora.client.base import get_job_page, synthesize_url
 from apache.aurora.client.cli import (
     EXIT_COMMAND_FAILURE,
     EXIT_INVALID_CONFIGURATION,
@@ -119,7 +121,7 @@ class CreateJobCommand(Verb):
     context.log_response_and_raise(resp, err_code=EXIT_COMMAND_FAILURE,
                                    err_msg="Job creation failed due to error:")
     if context.options.open_browser:
-      context.open_job_page(api, context.options.jobspec)
+      webbrowser.open_new_tab(get_job_page(api, context.options.jobspec))
     if context.options.wait_until == "RUNNING":
       JobMonitor(api.scheduler_proxy, config.job_key()).wait_until(JobMonitor.running_or_finished)
     elif context.options.wait_until == "FINISHED":
@@ -133,7 +135,7 @@ class CreateJobCommand(Verb):
       return EXIT_COMMAND_FAILURE
     else:
       context.print_out("Job create succeeded: job url=%s" %
-                        context.get_job_page(api, context.options.jobspec))
+                        get_job_page(api, context.options.jobspec))
     return EXIT_OK
 
 
@@ -372,7 +374,7 @@ class KillCommand(AbstractKillCommand):
     else:
       self.kill_in_batches(context, job, instances_arg)
     if context.options.open_browser:
-      context.open_job_page(api, context.options.jobspec)
+      webbrowser.open_new_tab(get_job_page(api, job))
     context.print_out("Job kill succeeded")
     return EXIT_OK
 
@@ -401,7 +403,7 @@ class KillAllJobCommand(AbstractKillCommand):
     else:
       self.kill_in_batches(context, job, None)
     if context.options.open_browser:
-      context.open_job_page(api, job)
+      webbrowser.open_new_tab(get_job_page(api, job))
     context.print_out("Job killall succeeded")
     return EXIT_OK
 
@@ -446,7 +448,10 @@ class OpenCommand(Verb):
     while len(key_parts) < 4:
       key_parts.append(None)
     (cluster, role, env, name) = key_parts
-    context.open_scheduler_page(cluster, role, env, name)
+    api = context.get_api(cluster)
+    webbrowser.open_new_tab(
+      synthesize_url(api.scheduler_proxy.scheduler_client().url, role, env, name)
+    )
     return EXIT_OK
 
 
@@ -510,7 +515,7 @@ class RestartCommand(Verb):
                                    err_msg="Error restarting job %s:" % str(job))
     context.print_out("Job %s restarted successfully" % str(job))
     if context.options.open_browser:
-      context.open_job_page(api, context.options.jobspec)
+      webbrowser.open_new_tab(get_job_page(api, job))
     return EXIT_OK
 
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/e8e1caaa/src/main/python/apache/aurora/client/cli/update.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/update.py b/src/main/python/apache/aurora/client/cli/update.py
index 5f98fc2..9f190a6 100644
--- a/src/main/python/apache/aurora/client/cli/update.py
+++ b/src/main/python/apache/aurora/client/cli/update.py
@@ -19,7 +19,7 @@ import json
 import textwrap
 from collections import namedtuple
 
-from apache.aurora.client.base import combine_messages
+from apache.aurora.client.base import combine_messages, get_update_page
 from apache.aurora.client.cli import (
     EXIT_API_ERROR,
     EXIT_COMMAND_FAILURE,
@@ -160,7 +160,10 @@ class StartUpdate(Verb):
         err_msg="Failed to start update due to error:")
 
     if resp.result:
-      url = context.get_update_page(api, config.cluster(), resp.result.startJobUpdateResult.key)
+      url = get_update_page(
+        api,
+        AuroraJobKey.from_thrift(config.cluster(), resp.result.startJobUpdateResult.key.job),
+        resp.result.startJobUpdateResult.key.id)
       context.print_out(self.UPDATE_MSG_TEMPLATE % url)
     else:
       context.print_out(combine_messages(resp))

http://git-wip-us.apache.org/repos/asf/aurora/blob/e8e1caaa/src/test/python/apache/aurora/client/cli/test_create.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/test_create.py b/src/test/python/apache/aurora/client/cli/test_create.py
index 57970c4..4de0ecb 100644
--- a/src/test/python/apache/aurora/client/cli/test_create.py
+++ b/src/test/python/apache/aurora/client/cli/test_create.py
@@ -15,7 +15,7 @@
 import contextlib
 
 import pytest
-from mock import create_autospec, Mock, patch
+from mock import call, create_autospec, Mock, patch
 from twitter.common.contextutil import temporary_file
 
 from apache.aurora.client.cli import (
@@ -174,7 +174,9 @@ class TestClientCreateCommand(AuroraClientCommandTest):
 
       self.assert_create_job_called(api)
       self.assert_scheduler_called(api, mock_query, 2)
-      assert mock_context.showed_urls == ["http://something_or_other/scheduler/bozo/test/hello"]
+      assert self.mock_webbrowser.mock_calls == [
+          call("http://something_or_other/scheduler/bozo/test/hello")
+      ]
 
   def test_create_job_delayed(self):
     """Run a test of the "create" command against a mocked-out API:

http://git-wip-us.apache.org/repos/asf/aurora/blob/e8e1caaa/src/test/python/apache/aurora/client/cli/test_cron.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/test_cron.py b/src/test/python/apache/aurora/client/cli/test_cron.py
index 9fa176e..f3c522e 100644
--- a/src/test/python/apache/aurora/client/cli/test_cron.py
+++ b/src/test/python/apache/aurora/client/cli/test_cron.py
@@ -16,9 +16,10 @@
 
 import contextlib
 
-from mock import patch
+from mock import call, patch
 from twitter.common.contextutil import temporary_file
 
+from apache.aurora.client.base import get_job_page
 from apache.aurora.client.cli import (
     EXIT_API_ERROR,
     EXIT_COMMAND_FAILURE,
@@ -54,7 +55,7 @@ class TestCronNoun(AuroraClientCommandTest):
       assert isinstance(api.schedule_cron.call_args[0][0], AuroraConfig)
 
       # The last text printed out to the user should contain a url to the job
-      assert mock_context.get_job_page(api, self.TEST_JOBKEY) in mock_context.out[-1]
+      assert get_job_page(api, self.TEST_JOBKEY) in mock_context.out[-1]
 
   def test_schedule_failed(self):
     mock_context = FakeAuroraCommandContext()
@@ -144,7 +145,9 @@ class TestCronNoun(AuroraClientCommandTest):
       result = cmd.execute(['cron', 'start', self.TEST_JOBSPEC, '--open-browser'])
       assert result == EXIT_OK
       api.start_cronjob.assert_called_once_with(self.TEST_JOBKEY, config=None)
-      assert mock_context.showed_urls == ["http://something_or_other/scheduler/bozo/test/hello"]
+      assert self.mock_webbrowser.mock_calls == [
+          call("http://something_or_other/scheduler/bozo/test/hello")
+      ]
 
   @classmethod
   def _create_getjobs_response(cls):

http://git-wip-us.apache.org/repos/asf/aurora/blob/e8e1caaa/src/test/python/apache/aurora/client/cli/test_kill.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/test_kill.py b/src/test/python/apache/aurora/client/cli/test_kill.py
index 69d8402..7d25860 100644
--- a/src/test/python/apache/aurora/client/cli/test_kill.py
+++ b/src/test/python/apache/aurora/client/cli/test_kill.py
@@ -140,6 +140,27 @@ class TestKillCommand(AuroraClientCommandTest):
     command.execute(fake_context)
     assert fake_context.get_err()[0] == "No tasks to kill found for job cluster/role/env/job"
 
+  def test_kill_opens_url(self):
+    """Verify the kill commands opens the job page if requested"""
+    command = KillCommand()
+
+    jobkey = AuroraJobKey("cluster", "role", "env", "job")
+
+    mock_options = mock_verb_options(command)
+    mock_options.instance_spec = TaskInstanceKey(jobkey, [1])
+    mock_options.open_browser = True
+
+    fake_context = FakeAuroraCommandContext()
+    fake_context.set_options(mock_options)
+
+    fake_context.add_expected_query_result(AuroraClientCommandTest.create_empty_task_result())
+
+    command.execute(fake_context)
+
+    assert self.mock_webbrowser.mock_calls == [
+        call("http://something_or_other/scheduler/role/env/job")
+    ]
+
 
 class TestClientKillCommand(AuroraClientCommandTest):
   @classmethod

http://git-wip-us.apache.org/repos/asf/aurora/blob/e8e1caaa/src/test/python/apache/aurora/client/cli/test_open.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/test_open.py b/src/test/python/apache/aurora/client/cli/test_open.py
index 92f9c3d..d3dd562 100644
--- a/src/test/python/apache/aurora/client/cli/test_open.py
+++ b/src/test/python/apache/aurora/client/cli/test_open.py
@@ -12,7 +12,7 @@
 # limitations under the License.
 #
 
-from mock import patch
+from mock import call, patch
 
 from apache.aurora.client.cli import EXIT_OK
 from apache.aurora.client.cli.client import AuroraCommandLine
@@ -27,7 +27,7 @@ class TestClientOpenCommand(AuroraClientCommandTest):
     with patch('apache.aurora.client.cli.jobs.Job.create_context', return_value=mock_context):
       cmd = AuroraCommandLine()
       result = cmd.execute(['job', 'open', 'west'])
-      assert mock_context.showed_urls == ['http://something_or_other/scheduler']
+      assert self.mock_webbrowser.mock_calls == [call("http://something_or_other/scheduler")]
       assert result == EXIT_OK
 
   def test_open_role(self):
@@ -35,7 +35,7 @@ class TestClientOpenCommand(AuroraClientCommandTest):
     with patch('apache.aurora.client.cli.jobs.Job.create_context', return_value=mock_context):
       cmd = AuroraCommandLine()
       result = cmd.execute(['job', 'open', 'west/bozo'])
-      assert mock_context.showed_urls == ['http://something_or_other/scheduler/bozo']
+      assert self.mock_webbrowser.mock_calls == [call("http://something_or_other/scheduler/bozo")]
       assert result == EXIT_OK
 
   def test_open_env(self):
@@ -43,7 +43,9 @@ class TestClientOpenCommand(AuroraClientCommandTest):
     with patch('apache.aurora.client.cli.jobs.Job.create_context', return_value=mock_context):
       cmd = AuroraCommandLine()
       result = cmd.execute(['job', 'open', 'west/bozo/devel'])
-      assert mock_context.showed_urls == ['http://something_or_other/scheduler/bozo/devel']
+      assert self.mock_webbrowser.mock_calls == [
+          call("http://something_or_other/scheduler/bozo/devel")
+      ]
       assert result == EXIT_OK
 
   def test_open_job(self):
@@ -51,7 +53,9 @@ class TestClientOpenCommand(AuroraClientCommandTest):
     with patch('apache.aurora.client.cli.jobs.Job.create_context', return_value=mock_context):
       cmd = AuroraCommandLine()
       result = cmd.execute(['job', 'open', 'west/bozo/devel/foo'])
-      assert mock_context.showed_urls == ['http://something_or_other/scheduler/bozo/devel/foo']
+      assert self.mock_webbrowser.mock_calls == [
+          call("http://something_or_other/scheduler/bozo/devel/foo")
+      ]
       assert result == EXIT_OK
 
   def test_open_noparam(self):
@@ -59,4 +63,4 @@ class TestClientOpenCommand(AuroraClientCommandTest):
     with patch('apache.aurora.client.cli.jobs.Job.create_context', return_value=mock_context):
       cmd = AuroraCommandLine()
       self.assertRaises(SystemExit, cmd.execute, (['job', 'open']))
-      assert mock_context.showed_urls == []
+      assert self.mock_webbrowser.mock_calls == []

http://git-wip-us.apache.org/repos/asf/aurora/blob/e8e1caaa/src/test/python/apache/aurora/client/cli/test_restart.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/test_restart.py b/src/test/python/apache/aurora/client/cli/test_restart.py
index fb5491d..488c6c9 100644
--- a/src/test/python/apache/aurora/client/cli/test_restart.py
+++ b/src/test/python/apache/aurora/client/cli/test_restart.py
@@ -15,7 +15,7 @@ import contextlib
 import functools
 
 import pytest
-from mock import create_autospec, patch
+from mock import call, create_autospec, patch
 from twitter.common.contextutil import temporary_file
 
 from apache.aurora.client.api import UpdaterConfig
@@ -78,6 +78,27 @@ class TestRestartJobCommand(AuroraClientCommandTest):
       command.execute(fake_context)
       assert e.message == "Invalid instance parameter: [1]"
 
+  def test_restart_opens_url(self):
+    command = RestartCommand()
+
+    jobkey = AuroraJobKey("cluster", "role", "env", "job")
+    mock_options = mock_verb_options(command)
+    mock_options.instance_spec = TaskInstanceKey(jobkey, None)
+    mock_options.strict = True
+    mock_options.open_browser = True
+
+    fake_context = FakeAuroraCommandContext()
+    fake_context.set_options(mock_options)
+    fake_api = fake_context.fake_api
+
+    fake_api.restart.return_value = AuroraClientCommandTest.create_simple_success_response()
+
+    command.execute(fake_context)
+
+    assert self.mock_webbrowser.mock_calls == [
+        call("http://something_or_other/scheduler/role/env/job")
+    ]
+
 
 class TestRestartCommand(AuroraClientCommandTest):
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/e8e1caaa/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 291186b..f856fcc 100644
--- a/src/test/python/apache/aurora/client/cli/util.py
+++ b/src/test/python/apache/aurora/client/cli/util.py
@@ -15,7 +15,7 @@
 import textwrap
 import unittest
 
-from mock import create_autospec, Mock
+from mock import create_autospec, Mock, patch
 
 from apache.aurora.client.cli.context import AuroraCommandContext
 from apache.aurora.client.hooks.hooked_api import HookedAuroraClientAPI
@@ -63,7 +63,6 @@ class FakeAuroraCommandContext(AuroraCommandContext):
     self.status = []
     self.fake_api = self.create_mock_api()
     self.task_result = []
-    self.showed_urls = []
     self.out = []
     self.err = []
 
@@ -98,9 +97,6 @@ class FakeAuroraCommandContext(AuroraCommandContext):
   def get_err(self):
     return self.err
 
-  def open_page(self, url):
-    self.showed_urls.append(url)
-
   def add_expected_status_query_result(self, expected_result):
     self.add_task_result(expected_result)
     self.fake_api.check_status.side_effect = self.task_result
@@ -121,6 +117,11 @@ class FakeAuroraCommandContext(AuroraCommandContext):
 class AuroraClientCommandTest(unittest.TestCase):
   FAKE_TIME = 42131
 
+  def setUp(self):
+    patcher = patch('webbrowser.open_new_tab')
+    self.mock_webbrowser = patcher.start()
+    self.addCleanup(patcher.stop)
+
   @classmethod
   def create_blank_response(cls, code, msg):
     return Response(