airflow-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From bo...@apache.org
Subject incubator-airflow git commit: [AIRFLOW-1731] Set pythonpath for logging
Date Fri, 27 Oct 2017 14:03:04 GMT
Repository: incubator-airflow
Updated Branches:
  refs/heads/master 2abead704 -> 635ab01a7


[AIRFLOW-1731] Set pythonpath for logging

Before initializing the logging framework, we want
to set the python
path so the logging config can be found.

Closes #2721 from Fokko/AIRFLOW-1731-import-
pythonpath


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

Branch: refs/heads/master
Commit: 635ab01a76d2187738b9b11f1b06c31e7b7dcf33
Parents: 2abead7
Author: Fokko Driesprong <fokkodriesprong@godatadriven.com>
Authored: Fri Oct 27 16:02:56 2017 +0200
Committer: Bolke de Bruin <bolke@xs4all.nl>
Committed: Fri Oct 27 16:02:56 2017 +0200

----------------------------------------------------------------------
 UPDATING.md               | 20 +++++++++++---------
 airflow/logging_config.py | 13 +++++++++++++
 docs/integration.rst      |  5 ++---
 3 files changed, 26 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/635ab01a/UPDATING.md
----------------------------------------------------------------------
diff --git a/UPDATING.md b/UPDATING.md
index ebcb5cd..6abcaf7 100644
--- a/UPDATING.md
+++ b/UPDATING.md
@@ -17,7 +17,7 @@ assists people when migrating to a new version.
 The logging structure of Airflow has been rewritten to make configuration easier and the
logging system more transparent.
 
 #### A quick recap about logging
- 
+
 A logger is the entry point into the logging system. Each logger is a named bucket to which
messages can be written for processing. A logger is configured to have a log level. This log
level describes the severity of the messages that the logger will handle. Python defines the
following log levels: DEBUG, INFO, WARNING, ERROR or CRITICAL.
 
 Each message that is written to the logger is a Log Record. Each log record also has a log
level indicating the severity of that specific message. A log record can also contain useful
metadata that describes the event that is being logged. This can include details such as a
stack trace or an error code.
@@ -39,7 +39,9 @@ The main benefit is easier configuration of the logging by setting a single
cent
 logging_config_class = my.path.default_local_settings.LOGGING_CONFIG
 ```
 
-The logging configuration file that contains the configuration needs te on the  the `PYTHONPATH`,
for example in `~/airflow/dags` or `~/airflow/plugins`. These directories are loaded by default,
of course you are free to add a directory to the `PYTHONPATH`, this might be handy when you
have the config in another directory or you mount a volume in case of Docker. As an example
you can start from `airflow.config_templates.airflow_local_settings.LOGGING_CONFIG`:
+The logging configuration file needs to be on the `PYTHONPATH`, for example `$AIRFLOW_HOME/config`.
This directory is loaded by default. Of course you are free to add any directory to the `PYTHONPATH`,
this might be handy when you have the config in another directory or you mount a volume in
case of Docker. 
+
+You can take the config from `airflow/config_templates/airflow_local_settings.py` as a starting
point. Copy the contents to `${AIRFLOW_HOME}/config/airflow_local_settings.py`,  and alter
the config as you like.
 
 ```
 LOGGING_CONFIG = {
@@ -106,19 +108,19 @@ Furthermore, this change also simplifies logging within the DAG itself:
 
 ```
 root@ae1bc863e815:/airflow# python
-Python 3.6.2 (default, Sep 13 2017, 14:26:54) 
+Python 3.6.2 (default, Sep 13 2017, 14:26:54)
 [GCC 4.9.2] on linux
 Type "help", "copyright", "credits" or "license" for more information.
 >>> from airflow.settings import *
->>> 
+>>>
 >>> from datetime import datetime
 >>> from airflow import DAG
 >>> from airflow.operators.dummy_operator import DummyOperator
->>> 
+>>>
 >>> dag = DAG('simple_dag', start_date=datetime(2017, 9, 1))
->>> 
+>>>
 >>> task = DummyOperator(task_id='task_1', dag=dag)
->>> 
+>>>
 >>> task.log.error('I want to say something..')
 [2017-09-25 20:17:04,927] {<stdin>:1} ERROR - I want to say something..
 ```
@@ -132,9 +134,9 @@ The `file_task_handler` logger is more flexible. You can change the default
form
 If you are logging to Google cloud storage, please see the [Google cloud platform documentation](https://airflow.incubator.apache.org/integration.html#gcp-google-cloud-platform)
for logging instructions.
 
 If you are using S3, the instructions should be largely the same as the Google cloud platform
instructions above. You will need a custom logging config. The `REMOTE_BASE_LOG_FOLDER` configuration
key in your airflow config has been removed, therefore you will need to take the following
steps:
- - Copy the logging configuration from [`airflow/config_templates/airflow_logging_settings.py`](https://github.com/apache/incubator-airflow/blob/master/airflow/config_templates/airflow_local_settings.py)
and copy it. 
+ - Copy the logging configuration from [`airflow/config_templates/airflow_logging_settings.py`](https://github.com/apache/incubator-airflow/blob/master/airflow/config_templates/airflow_local_settings.py)
and copy it.
  - Place it in a directory inside the Python import path `PYTHONPATH`. If you are using Python
2.7, ensuring that any `__init__.py` files exist so that it is importable.
- - Update the config by setting the path of `REMOTE_BASE_LOG_FOLDER` explicitly in the config.
The `REMOTE_BASE_LOG_FOLDER` key is not used anymore. 
+ - Update the config by setting the path of `REMOTE_BASE_LOG_FOLDER` explicitly in the config.
The `REMOTE_BASE_LOG_FOLDER` key is not used anymore.
  - Set the `logging_config_class` to the filename and dict. For example, if you place `custom_logging_config.py`
on the base of your pythonpath, you will need to set `logging_config_class = custom_logging_config.LOGGING_CONFIG`
in your config as Airflow 1.8.
 
 ### New Features

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/635ab01a/airflow/logging_config.py
----------------------------------------------------------------------
diff --git a/airflow/logging_config.py b/airflow/logging_config.py
index d8c288a..0d3893a 100644
--- a/airflow/logging_config.py
+++ b/airflow/logging_config.py
@@ -13,6 +13,7 @@
 # limitations under the License.
 #
 import logging
+import os
 import sys
 from logging.config import dictConfig
 
@@ -23,9 +24,21 @@ from airflow.utils.module_loading import import_string
 log = logging.getLogger(__name__)
 
 
+def prepare_classpath():
+    config_path = os.path.join(conf.get('core', 'airflow_home'), 'config')
+    config_path = os.path.expanduser(config_path)
+
+    if config_path not in sys.path:
+        sys.path.append(config_path)
+
+
 def configure_logging():
     logging_class_path = ''
     try:
+        # Prepare the classpath so we are sure that the config folder
+        # is on the python classpath and it is reachable
+        prepare_classpath()
+
         logging_class_path = conf.get('core', 'logging_config_class')
     except AirflowConfigException:
         log.debug('Could not find key logging_config_class in config')

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/635ab01a/docs/integration.rst
----------------------------------------------------------------------
diff --git a/docs/integration.rst b/docs/integration.rst
index cd6cc68..4887486 100644
--- a/docs/integration.rst
+++ b/docs/integration.rst
@@ -185,13 +185,12 @@ Operators are in the contrib section. Meaning that they have a *beta*
status, me
 they can have breaking changes between minor releases.
 
 Logging
-''''''''
+'''''''
 
 Airflow can be configured to read and write task logs in Google cloud storage.
 Follow the steps below to enable Google cloud storage logging.
 
-#. Airlfow's logging system requires a custom .py file to be located in the ``PYTHONPATH``,
so that it's importable from Airflow. Start by creating a directory to store the config file.
``$AIRFLOW_HOME/config`` is recommended.
-#. Set ``PYTHONPATH=$PYTHONPATH:<AIRFLOW_HOME>/config`` in the Airflow environment.
If using Supervisor, you can set this in the ``supervisord.conf`` environment parameter. If
not, you can export ``PYTHONPATH`` using your preferred method.
+#. Airflow's logging system requires a custom .py file to be located in the ``PYTHONPATH``,
so that it's importable from Airflow. Start by creating a directory to store the config file.
``$AIRFLOW_HOME/config`` is recommended.
 #. Create empty files called ``$AIRFLOW_HOME/config/log_config.py`` and ``$AIRFLOW_HOME/config/__init__.py``.
 #. Copy the contents of ``airflow/config_templates/airflow_local_settings.py`` into the ``log_config.py``
file that was just created in the step above.
 #. Customize the following portions of the template:


Mime
View raw message