[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
This commit is contained in:
Fokko Driesprong 2017-10-27 16:02:56 +02:00 коммит произвёл Bolke de Bruin
Родитель 2abead7049
Коммит 635ab01a76
3 изменённых файлов: 26 добавлений и 12 удалений

Просмотреть файл

@ -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 = {

Просмотреть файл

@ -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')

Просмотреть файл

@ -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: