You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by mc...@apache.org on 2014/03/25 21:22:31 UTC

git commit: Implement client command logging with a unique token.

Repository: incubator-aurora
Updated Branches:
  refs/heads/master 9ff7264f4 -> a4d30e8e5


Implement client command logging with a unique token.

Adding a centralized logger interface for clientv2.

For the aurora analytics work, we need some hooks for logging. We'd
like all client logging to go through a common interface, and for
client logs that might be sent to the centralized logging service to
be tagged with identifiers and userids.

This change implements a simple minimal logging function that's used
by clientv2 logging calls, and which attaches a universally unique
client identifier to each log message.

Bugs closed: aurora-276

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


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

Branch: refs/heads/master
Commit: a4d30e8e5010679a73e935284fbefea19e9e995c
Parents: 9ff7264
Author: Mark Chu-Carroll <mc...@twopensource.com>
Authored: Tue Mar 25 16:20:49 2014 -0400
Committer: Mark Chu-Carroll <mc...@twitter.com>
Committed: Tue Mar 25 16:20:49 2014 -0400

----------------------------------------------------------------------
 .../python/apache/aurora/client/cli/__init__.py |  59 +++++++-
 .../python/apache/aurora/client/cli/context.py  |  19 +--
 .../python/apache/aurora/client/cli/jobs.py     |   7 +
 src/test/python/apache/aurora/client/cli/BUILD  |  13 ++
 .../apache/aurora/client/cli/test_logging.py    | 134 +++++++++++++++++++
 5 files changed, 212 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/a4d30e8e/src/main/python/apache/aurora/client/cli/__init__.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/__init__.py b/src/main/python/apache/aurora/client/cli/__init__.py
index 4a6a7ee..a1063e1 100644
--- a/src/main/python/apache/aurora/client/cli/__init__.py
+++ b/src/main/python/apache/aurora/client/cli/__init__.py
@@ -34,7 +34,10 @@ from __future__ import print_function
 
 from abc import abstractmethod
 import argparse
+import getpass
+import logging
 import sys
+from uuid import uuid1
 
 
 # Constants for standard return codes.
@@ -50,6 +53,24 @@ EXIT_API_ERROR = 10
 EXIT_UNKNOWN_ERROR = 20
 
 
+# Set up a logging call that adds a unique identifier for this invocation
+# of the client. Log messages sent via this call will contain two additional
+# fields in the log record: "clientid", which contains a UUID for the client
+# invocation, and "user", which contains the username of the user who invoked
+# the client.
+
+logger = logging.getLogger('aurora_client')
+CLIENT_ID = uuid1()
+
+
+def print_aurora_log(sev, msg, *args, **kwargs):
+  extra = kwargs.get('extra', {})
+  extra['clientid'] = CLIENT_ID
+  extra['user'] = getpass.getuser()
+  kwargs['extra'] = extra
+  logger.log(sev, msg, *args, **kwargs)
+
+
 class Context(object):
   class Error(Exception): pass
 
@@ -60,6 +81,7 @@ class Context(object):
       super(Context.CommandError, self).__init__(msg)
       self.msg = msg
       self.code = code
+      self.options = None
 
   @classmethod
   def exit(cls, code, msg):
@@ -71,6 +93,31 @@ class Context(object):
     """
     self.options = options
 
+  def print_out(self, msg, indent=0):
+    """Prints output to standard out with indent.
+    For debugging purposes, it's nice to be able to patch this and capture output.
+    """
+    indent_str = ' ' * indent
+    lines = msg.split('\n')
+    for line in lines:
+      print('%s%s' % (indent_str, line))
+
+  def print_err(self, msg, indent=0):
+    """Prints output to standard error, with an indent."""
+    indent_str = ' ' * indent
+    lines = msg.split('\n')
+    for line in lines:
+      print('%s%s' % (indent_str, line), file=sys.stderr)
+
+  def print_log(self, severity, msg, *args, **kwargs):
+    """Print a message to a log.
+    Logging with this method is intended for generating output for aurora developers/maintainers.
+    Log output isn't for users - information much more detailed than users want may be logged.
+    Logs generated for clients of a cluster may be gathered in a centralized database by the
+    aurora admins for that cluster.
+    """
+    print_aurora_log(severity, msg, *args, **kwargs)
+
 
 class ConfigurationPlugin(object):
   """A component that can be plugged in to a command-line.
@@ -233,6 +280,7 @@ class CommandLine(object):
     :param args: the command-line arguments for the command. This only includes arguments
         that should be parsed by the application; it does not include sys.argv[0].
     """
+    print_aurora_log(logging.INFO, 'Command=(%s)', args)
     nouns = self.registered_nouns
     if args[0] == 'help':
       return self.help_cmd(args[1:])
@@ -250,8 +298,14 @@ class CommandLine(object):
       print('Error in configuration plugin: %s' % c.msg, file=sys.stderr)
       return c.code
     try:
-      return noun.execute(context)
+      result = noun.execute(context)
+      if result == EXIT_OK:
+        print_aurora_log(logging.INFO, 'Command terminated successfully')
+      else:
+        print_aurora_log(logging.INFO, 'Commmand terminated with error code %s', result)
+      return result
     except Context.CommandError as c:
+      print_aurora_log(logging.INFO, 'Error executing command: %s', c.msg)
       print('Error executing command: %s' % c.msg, file=sys.stderr)
       return c.code
 
@@ -309,7 +363,7 @@ class Noun(AuroraCommand):
     if context.options.verb not in self.verbs:
       raise self.InvalidVerbException('Noun %s does not have a verb %s' %
           (self.name, context.options.verb))
-    self.verbs[context.options.verb].execute(context)
+    return self.verbs[context.options.verb].execute(context)
 
 
 class Verb(AuroraCommand):
@@ -349,4 +403,3 @@ class Verb(AuroraCommand):
 
   def execute(self, context):
     pass
-

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/a4d30e8e/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 dad4fcb..c07f54a 100644
--- a/src/main/python/apache/aurora/client/cli/context.py
+++ b/src/main/python/apache/aurora/client/cli/context.py
@@ -17,6 +17,7 @@
 from __future__ import print_function
 from collections import namedtuple
 from fnmatch import fnmatch
+import logging
 import sys
 
 from apache.aurora.common.clusters import CLUSTERS
@@ -73,22 +74,6 @@ class AuroraCommandContext(Context):
     except Exception as e:
       raise self.CommandError(EXIT_INVALID_CONFIGURATION, 'Error loading configuration: %s' % e)
 
-  def print_out(self, msg, indent=0):
-    """Prints output. For debugging purposes, it's nice to be able to patch this
-    and capture output.
-    """
-    indent_str = ' ' * indent
-    lines = msg.split('\n')
-    for line in lines:
-      print('%s%s' % (indent_str, line))
-
-  def print_err(self, msg, indent=0):
-    """Prints output to standard error."""
-    indent_str = ' ' * indent
-    lines = msg.split('\n')
-    for line in lines:
-      print('%s%s' % (indent_str, line), file=sys.stderr)
-
   def open_page(self, url):
     import webbrowser
     webbrowser.open_new_tab(url)
@@ -99,7 +84,7 @@ class AuroraCommandContext(Context):
         jobkey.env, jobkey.name))
 
   def check_and_log_response(self, resp):
-    log.info('Response from scheduler: %s (message: %s)'
+    self.print_log(logging.INFO, 'Response from scheduler: %s (message: %s)'
         % (ResponseCode._VALUES_TO_NAMES[resp.responseCode], resp.message))
     if resp.responseCode != ResponseCode.OK:
       raise self.CommandError(EXIT_API_ERROR, resp.message)

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/a4d30e8e/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 3a68cf8..290a8ba 100644
--- a/src/main/python/apache/aurora/client/cli/jobs.py
+++ b/src/main/python/apache/aurora/client/cli/jobs.py
@@ -88,6 +88,7 @@ class CancelUpdateCommand(Verb):
         if context.options.config_file else None)
     resp = api.cancel_update(context.options.jobspec, config=config)
     context.check_and_log_response(resp)
+    return EXIT_OK
 
 
 class CreateJobCommand(Verb):
@@ -269,6 +270,7 @@ the parsed configuration."""
       for line in process.cmdline().get().splitlines():
         context.print_out(line, indent=4)
       context.print_out()
+    return EXIT_OK
 
 
 class KillJobCommand(Verb):
@@ -295,6 +297,7 @@ class KillJobCommand(Verb):
       return EXIT_INVALID_PARAMETER
     if context.options.open_browser:
       context.open_job_page(api, context.options.jobspec)
+    return EXIT_OK
 
 
 class ListJobsCommand(Verb):
@@ -316,6 +319,7 @@ class ListJobsCommand(Verb):
       context.print_out('%s/%s/%s/%s' % (j.cluster, j.role, j.env, j.name))
     result = self.get_status_for_jobs(jobs, context)
     context.print_out(result)
+    return EXIT_OK
 
 
 class RestartCommand(Verb):
@@ -360,6 +364,7 @@ Restarts are fully controlled client-side, so aborting halts the restart."""
     context.check_and_log_response(resp)
     if context.options.open_browser:
       context.open_job_page(api, context.options.jobspec)
+    return EXIT_OK
 
 
 class StatusCommand(Verb):
@@ -446,6 +451,7 @@ The jobspec parameter can omit parts of the jobkey, or use shell-style globs."""
     jobs = context.get_jobs_matching_key(context.options.jobspec)
     result = self.get_status_for_jobs(jobs, context)
     context.print_out(result)
+    return EXIT_OK
 
 
 class UpdateCommand(Verb):
@@ -507,6 +513,7 @@ to preview what changes will take effect.
         context.options.instances)
     if resp.responseCode != ResponseCode.OK:
       raise context.CommandError(EXIT_COMMAND_FAILURE, 'Update failed: %s' % resp.message)
+    return EXIT_OK
 
 
 class Job(Noun):

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/a4d30e8e/src/test/python/apache/aurora/client/cli/BUILD
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/BUILD b/src/test/python/apache/aurora/client/cli/BUILD
index a48284c..d93a2df 100644
--- a/src/test/python/apache/aurora/client/cli/BUILD
+++ b/src/test/python/apache/aurora/client/cli/BUILD
@@ -20,6 +20,7 @@ python_test_suite(
     pants(':bridge'),
     pants(':help'),
     pants(':job'),
+    pants(':logging'),
     pants(':plugins'),
     pants(':quota'),
     pants(':sla'),
@@ -48,6 +49,18 @@ python_tests(
 )
 
 python_tests(
+  name = 'logging',
+  sources = [ 'test_logging.py' ],
+  dependencies = [
+    pants(':util'),
+    pants('3rdparty/python:mock'),
+    pants('3rdparty/python:twitter.common.contextutil'),
+    pants('src/main/python/apache/aurora/client/cli'),
+    pants('src/main/python/apache/aurora/client/cli:client'),
+  ]
+)
+
+python_tests(
   name = 'bridge',
   sources = [ 'test_bridge.py' ],
   dependencies = [

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/a4d30e8e/src/test/python/apache/aurora/client/cli/test_logging.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/test_logging.py b/src/test/python/apache/aurora/client/cli/test_logging.py
new file mode 100644
index 0000000..b6241b8
--- /dev/null
+++ b/src/test/python/apache/aurora/client/cli/test_logging.py
@@ -0,0 +1,134 @@
+#
+# Copyright 2013 Apache Software Foundation
+#
+# 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 logging
+from logging import Handler
+
+from twitter.common.contextutil import temporary_file
+
+from gen.apache.aurora.ttypes import (
+    AssignedTask,
+    Identity,
+    ScheduledTask,
+    ScheduleStatus,
+    ScheduleStatusResult,
+    TaskEvent,
+    TaskQuery,
+)
+
+from apache.aurora.client.cli.client import AuroraCommandLine
+from apache.aurora.client.cli.util import AuroraClientCommandTest, FakeAuroraCommandContext
+from apache.aurora.config import AuroraConfig
+from mock import Mock, patch
+
+
+class MockHandler(Handler):
+  def __init__(self):
+    Handler.__init__(self)
+    self.logs = []
+
+  def emit(self, record):
+    self.logs.append(record)
+
+
+class TestLogging(AuroraClientCommandTest):
+
+  @classmethod
+  def create_mock_task(cls, task_id, instance_id, initial_time, status):
+    mock_task = Mock(spec=ScheduledTask)
+    mock_task.assignedTask = Mock(spec=AssignedTask)
+    mock_task.assignedTask.taskId = task_id
+    mock_task.assignedTask.instanceId = instance_id
+    mock_task.status = status
+    mock_task_event = Mock(spec=TaskEvent)
+    mock_task_event.timestamp = initial_time
+    mock_task.taskEvents = [mock_task_event]
+    return mock_task
+
+  @classmethod
+  def create_mock_status_query_result(cls, scheduleStatus):
+    mock_query_result = cls.create_simple_success_response()
+    mock_query_result.result.scheduleStatusResult = Mock(spec=ScheduleStatusResult)
+    if scheduleStatus == ScheduleStatus.INIT:
+      # status query result for before job is launched.
+      mock_query_result.result.scheduleStatusResult.tasks = []
+    else:
+      mock_task_one = cls.create_mock_task('hello', 0, 1000, scheduleStatus)
+      mock_task_two = cls.create_mock_task('hello', 1, 1004, scheduleStatus)
+      mock_query_result.result.scheduleStatusResult.tasks = [mock_task_one, mock_task_two]
+    return mock_query_result
+
+  @classmethod
+  def create_mock_query(cls):
+    return TaskQuery(owner=Identity(role=cls.TEST_ROLE), environment=cls.TEST_ENV,
+        jobName=cls.TEST_JOB)
+
+  @classmethod
+  def get_createjob_response(cls):
+    # Then, we call api.create_job(config)
+    return cls.create_simple_success_response()
+
+  @classmethod
+  def get_failed_createjob_response(cls):
+    return cls.create_error_response()
+
+  @classmethod
+  def assert_create_job_called(cls, mock_api):
+    # Check that create_job was called exactly once, with an AuroraConfig parameter.
+    assert mock_api.create_job.call_count == 1
+    assert isinstance(mock_api.create_job.call_args_list[0][0][0], AuroraConfig)
+
+  @classmethod
+  def assert_scheduler_called(cls, mock_api, mock_query, num_queries):
+    assert mock_api.scheduler_proxy.getTasksStatus.call_count == num_queries
+    mock_api.scheduler_proxy.getTasksStatus.assert_called_with(mock_query)
+
+  def test_command_invocation_logging(self):
+    """Sets up a log handler, registers it with the logger, and then verifies that calls
+    to the client logging methods correctly get captured in the logs.
+    """
+    mock_log_handler = MockHandler()
+    logger = logging.getLogger('aurora_client')
+    logger.setLevel(logging.INFO)
+    logger.addHandler(mock_log_handler)
+    # We'll patch out create_context, which will give us a fake context
+    # object, and everything can be stubbed through that.
+    mock_context = FakeAuroraCommandContext()
+    with patch('apache.aurora.client.cli.jobs.Job.create_context', return_value=mock_context):
+      # After making the client, create sets up a job monitor.
+      # The monitor uses TaskQuery to get the tasks. It's called at least twice:once before
+      # the job is created, and once after. So we need to set up mocks for the query results.
+      mock_context.add_expected_status_query_result(
+        self.create_mock_status_query_result(ScheduleStatus.INIT))
+      mock_context.add_expected_status_query_result(
+        self.create_mock_status_query_result(ScheduleStatus.RUNNING))
+      api = mock_context.get_api('west')
+      api.create_job.return_value = self.get_createjob_response()
+
+      with temporary_file() as fp:
+        fp.write(self.get_valid_config())
+        fp.flush()
+        cmd = AuroraCommandLine()
+        cmd.execute(['job', 'create', '--wait-until=RUNNING', 'west/bozo/test/hello',
+            fp.name])
+
+      # Check that things were logged correctly:
+      # there should be two entries, with the clientid and username;
+      # the first entry should log the command being invoked.
+      assert ("'job', 'create', '--wait-until=RUNNING', 'west/bozo/test/hello'" in
+          mock_log_handler.logs[0].getMessage())
+      assert mock_log_handler.logs[0].clientid == mock_log_handler.logs[1].clientid
+      assert mock_log_handler.logs[0].user == mock_log_handler.logs[1].user