You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2019/12/05 18:27:47 UTC

[GitHub] [airflow] mik-laj commented on a change in pull request #6660: [AIRFLOW-6065] Add Stackdriver Task Handler

mik-laj commented on a change in pull request #6660: [AIRFLOW-6065] Add Stackdriver Task Handler
URL: https://github.com/apache/airflow/pull/6660#discussion_r354475809
 
 

 ##########
 File path: airflow/utils/log/stackdriver_task_handler.py
 ##########
 @@ -0,0 +1,286 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you 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.
+"""
+Handler that integrates with Stackdriver
+"""
+import logging
+from typing import Dict, List, Optional, Tuple, Type
+
+from cached_property import cached_property
+from google.api_core.gapic_v1.client_info import ClientInfo
+from google.cloud import logging as gcp_logging
+from google.cloud.logging.handlers.transports import BackgroundThreadTransport, Transport
+from google.cloud.logging.resource import Resource
+
+from airflow import version
+from airflow.models import TaskInstance
+
+DEFAULT_LOGGER_NAME = "airflow"
+_GLOBAL_RESOURCE = Resource(type="global", labels={})
+
+
+class StackdriverTaskHandler(logging.Handler):
+    """Handler that directly makes Stackdriver logging API calls.
+
+    This is a Python standard ``logging`` handler using that can be used to
+    route Python standard logging messages directly to the Stackdriver
+    Logging API.
+
+    It can also be used to save logs for executing tasks. To do this, you should set as a handler with
+    the name "tasks". In this case, it will also be used to read the log for display in Web UI.
+
+    This handler supports both an asynchronous and synchronous transport.
+
+    :param gcp_conn_id: Connection ID that will be used for authorization to the Google Cloud Platform.
+        If omitted, authorization based on `the Application Default Credentials
+        <https://cloud.google.com/docs/authentication/production#finding_credentials_automatically>`__ will
+        be used.
+    :type gcp_conn_id: str
+    :param name: the name of the custom log in Stackdriver Logging. Defaults
+        to 'airflow'. The name of the Python logger will be represented
+         in the ``python_logger`` field.
+    :type name: str
+    :param transport: Class for creating new transport objects. It should
+        extend from the base :class:`google.cloud.logging.handlers.Transport` type and
+        implement :meth`google.cloud.logging.handlers.Transport.send`. Defaults to
+        :class:`google.cloud.logging.handlers.BackgroundThreadTransport`. The other
+        option is :class:`google.cloud.logging.handlers.SyncTransport`.
+    :type transport: :class:`type`
+    :param resource: (Optional) Monitored resource of the entry, defaults
+                     to the global resource type.
+    :type resource: :class:`~google.cloud.logging.resource.Resource`
+    :param labels: (Optional) Mapping of labels for the entry.
+    :type labels: dict
+    """
+
+    LABEL_TASK_ID = "task_id"
+    LABEL_DAG_ID = "dag_id"
+    LABEL_EXECUTION_DATE = "execution_date"
+    LABEL_TRY_NUMBER = "try_number"
+
+    def __init__(
+        self,
+        gcp_conn_id: Optional[str] = None,
+        name: str = DEFAULT_LOGGER_NAME,
+        transport: Type[Transport] = BackgroundThreadTransport,
+        resource: Resource = _GLOBAL_RESOURCE,
+        labels: Optional[Dict[str, str]] = None,
+    ):
+        super().__init__()
+        self.gcp_conn_id = gcp_conn_id
+        self.name: str = name
+        self.transport_type: Type[Transport] = transport
+        self.resource: Resource = resource
+        self.labels: Optional[Dict[str, str]] = labels
+        self.task_instance_labels: Optional[Dict[str, str]] = {}
+
+    @cached_property
+    def _client(self) -> gcp_logging.Client:
+        """Google Cloud Library API clinet"""
+        if self.gcp_conn_id:
+            from airflow.gcp.hooks.base import GoogleCloudBaseHook
+
+            hook = GoogleCloudBaseHook(gcp_conn_id=self.gcp_conn_id)
+            credentials = hook._get_credentials()  # pylint: disable=protected-access
+        else:
+            # Use Application Default Credentials
+            credentials = None
+        client = gcp_logging.Client(
+            credentials=credentials,
+            client_info=ClientInfo(client_library_version='airflow_v' + version.version)
+        )
+        return client
+
+    @cached_property
+    def _transport(self) -> Transport:
+        """Object responsible for sending data to Stackdriver"""
+        return self.transport_type(self._client, self.name)
+
+    def emit(self, record: logging.LogRecord) -> None:
+        """Actually log the specified logging record.
+
+        :param record: The record to be logged.
+        :type record: logging.LogRecord
+        """
+        message = self.format(record)
+        labels: Optional[Dict[str, str]]
+        if self.labels and self.task_instance_labels:
+            labels = {}
+            labels.update(self.labels)
+            labels.update(self.task_instance_labels)
+        elif self.labels:
+            labels = self.labels
+        elif self.task_instance_labels:
+            labels = self.task_instance_labels
+        else:
+            labels = None
+        self._transport.send(record, message, resource=self.resource, labels=labels)
+
+    def set_context(self, task_instance: TaskInstance) -> None:
+        """
+        Configures the logger to add information with information about the current task
+
+        :param task_instance: Currently executed task
+        :type task_instance: TaskInstance
+        """
+        self.task_instance_labels = self._task_instance_to_labels(task_instance)
+
+    def read(
 
 Review comment:
   I would like to avoid modifying the API, but in this change I would like to implement the current Airflow interface. Currently in Airflow the process of saving and reading is combined in one class.  
   If you want, you can configure another handler to read data using ``task_log_reader `` option in ``airflow.cfg``.
   https://github.com/apache/airflow/blob/master/airflow/config_templates/default_airflow.cfg#L191-L193
   I agree that the log API needs improvement, but this is not the scope of this PR.
   
   Can you look at my comment that discusses API issues?
   https://github.com/apache/airflow/pull/5177#issuecomment-557788417
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services