Skip to content

Commit

Permalink
[AIRFLOW-6708] Set unique logger names (#7330)
Browse files Browse the repository at this point in the history
  • Loading branch information
mik-laj committed Feb 2, 2020
1 parent 1e576f1 commit cf14150
Show file tree
Hide file tree
Showing 56 changed files with 223 additions and 200 deletions.
6 changes: 6 additions & 0 deletions .pre-commit-config.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -243,6 +243,12 @@ repos:
entry: "from airflow\\.utils\\.db import.* (provide_session|create_session)"
files: \.py$
pass_filenames: true
- id: incorrect-use-of-LoggingMixin
language: pygrep
name: Make sure LoggingMixin is not used alone
entry: "LoggingMixin\\(\\)"
files: \.py$
pass_filenames: true
- id: build
name: Check if image build is needed
entry: ./scripts/ci/pre_commit_ci_build.sh
Expand Down
9 changes: 4 additions & 5 deletions airflow/api/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,13 @@
# specific language governing permissions and limitations
# under the License.
"""Authentication backend"""

import logging
from importlib import import_module

from airflow.configuration import conf
from airflow.exceptions import AirflowConfigException, AirflowException
from airflow.utils.log.logging_mixin import LoggingMixin

log = logging.getLogger(__name__)


class ApiAuth: # pylint: disable=too-few-public-methods
Expand All @@ -33,8 +34,6 @@ def __init__(self):

API_AUTH = ApiAuth()

LOG = LoggingMixin().log


def load_auth():
"""Loads authentication backend"""
Expand All @@ -47,7 +46,7 @@ def load_auth():
try:
API_AUTH.api_auth = import_module(auth_backend)
except ImportError as err:
LOG.critical(
log.critical(
"Cannot import %s for API authentication due to: %s",
auth_backend, err
)
Expand Down
15 changes: 7 additions & 8 deletions airflow/api/auth/backend/kerberos_auth.py
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@
# (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
# SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
"""Kerberos authentication module"""
import logging
import os
from functools import wraps
from socket import getfqdn
Expand All @@ -51,15 +52,13 @@
from requests_kerberos import HTTPKerberosAuth

from airflow.configuration import conf
from airflow.utils.log.logging_mixin import LoggingMixin

log = logging.getLogger(__name__)

# pylint: disable=c-extension-no-member
CLIENT_AUTH = HTTPKerberosAuth(service='airflow')


LOG = LoggingMixin().log


class KerberosService: # pylint: disable=too-few-public-methods
"""Class to keep information about the Kerberos Service initialized """
def __init__(self):
Expand All @@ -76,7 +75,7 @@ def init_app(app):
hostname = app.config.get('SERVER_NAME')
if not hostname:
hostname = getfqdn()
LOG.info("Kerberos: hostname %s", hostname)
log.info("Kerberos: hostname %s", hostname)

service = 'airflow'

Expand All @@ -86,12 +85,12 @@ def init_app(app):
os.environ['KRB5_KTNAME'] = conf.get('kerberos', 'keytab')

try:
LOG.info("Kerberos init: %s %s", service, hostname)
log.info("Kerberos init: %s %s", service, hostname)
principal = kerberos.getServerPrincipalDetails(service, hostname)
except kerberos.KrbError as err:
LOG.warning("Kerberos: %s", err)
log.warning("Kerberos: %s", err)
else:
LOG.info("Kerberos API: server is %s", principal)
log.info("Kerberos API: server is %s", principal)


def _unauthorized():
Expand Down
7 changes: 4 additions & 3 deletions airflow/api/common/experimental/delete_dag.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
# specific language governing permissions and limitations
# under the License.
"""Delete DAGs APIs."""
import logging

from sqlalchemy import or_

Expand All @@ -25,9 +26,10 @@
from airflow.models import DagModel, TaskFail
from airflow.models.serialized_dag import SerializedDagModel
from airflow.settings import STORE_SERIALIZED_DAGS
from airflow.utils.log.logging_mixin import LoggingMixin
from airflow.utils.session import provide_session

log = logging.getLogger(__name__)


@provide_session
def delete_dag(dag_id: str, keep_records_in_log: bool = True, session=None) -> int:
Expand All @@ -39,8 +41,7 @@ def delete_dag(dag_id: str, keep_records_in_log: bool = True, session=None) -> i
:param session: session used
:return count of deleted dags
"""
logger = LoggingMixin()
logger.log.info("Deleting DAG: %s", dag_id)
log.info("Deleting DAG: %s", dag_id)
dag = session.query(DagModel).filter(DagModel.dag_id == dag_id).first()
if dag is None:
raise DagNotFound("Dag id {} not found".format(dag_id))
Expand Down
21 changes: 11 additions & 10 deletions airflow/cli/commands/webserver_command.py
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
# under the License.

"""Webserver command"""
import logging
import os
import signal
import subprocess
Expand All @@ -27,13 +28,13 @@
import psutil
from daemon.pidfile import TimeoutPIDLockFile

from airflow import AirflowException, LoggingMixin, conf, settings
from airflow import AirflowException, conf, settings
from airflow.exceptions import AirflowWebServerTimeout
from airflow.utils import cli as cli_utils
from airflow.utils.cli import setup_locations, setup_logging
from airflow.www.app import cached_app, create_app

LOG = LoggingMixin().log
log = logging.getLogger(__name__)


def get_num_ready_workers_running(gunicorn_master_proc):
Expand Down Expand Up @@ -95,7 +96,7 @@ def wait_until_true(fn, timeout=0):

def start_refresh(gunicorn_master_proc):
batch_size = conf.getint('webserver', 'worker_refresh_batch_size')
LOG.debug('%s doing a refresh of %s workers', state, batch_size)
log.debug('%s doing a refresh of %s workers', state, batch_size)
sys.stdout.flush()
sys.stderr.flush()

Expand All @@ -120,14 +121,14 @@ def start_refresh(gunicorn_master_proc):

# Whenever some workers are not ready, wait until all workers are ready
if num_ready_workers_running < num_workers_running:
LOG.debug('%s some workers are starting up, waiting...', state)
log.debug('%s some workers are starting up, waiting...', state)
sys.stdout.flush()
time.sleep(1)

# Kill a worker gracefully by asking gunicorn to reduce number of workers
elif num_workers_running > num_workers_expected:
excess = num_workers_running - num_workers_expected
LOG.debug('%s killing %s workers', state, excess)
log.debug('%s killing %s workers', state, excess)

for _ in range(excess):
gunicorn_master_proc.send_signal(signal.SIGTTOU)
Expand All @@ -139,7 +140,7 @@ def start_refresh(gunicorn_master_proc):
# Start a new worker by asking gunicorn to increase number of workers
elif num_workers_running == num_workers_expected:
refresh_interval = conf.getint('webserver', 'worker_refresh_interval')
LOG.debug(
log.debug(
'%s sleeping for %ss starting doing a refresh...',
state, refresh_interval
)
Expand All @@ -148,7 +149,7 @@ def start_refresh(gunicorn_master_proc):

else:
# num_ready_workers_running == num_workers_running < num_workers_expected
LOG.error((
log.error((
"%s some workers seem to have died and gunicorn"
"did not restart them as expected"
), state)
Expand All @@ -158,8 +159,8 @@ def start_refresh(gunicorn_master_proc):
) < num_workers_expected:
start_refresh(gunicorn_master_proc)
except (AirflowWebServerTimeout, OSError) as err:
LOG.error(err)
LOG.error("Shutting down webserver")
log.error(err)
log.error("Shutting down webserver")
try:
gunicorn_master_proc.terminate()
gunicorn_master_proc.wait()
Expand Down Expand Up @@ -285,7 +286,7 @@ def monitor_gunicorn(gunicorn_master_proc):
gunicorn_master_proc_pid = int(file.read())
break
except OSError:
LOG.debug("Waiting for gunicorn's pid file to be created.")
log.debug("Waiting for gunicorn's pid file to be created.")
time.sleep(0.1)

gunicorn_master_proc = psutil.Process(gunicorn_master_proc_pid)
Expand Down
4 changes: 2 additions & 2 deletions airflow/config_templates/default_celery.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,18 +17,18 @@
# specific language governing permissions and limitations
# under the License.
"""Default celery configuration."""
import logging
import ssl

from airflow.configuration import conf
from airflow.exceptions import AirflowConfigException, AirflowException
from airflow.utils.log.logging_mixin import LoggingMixin


def _broker_supports_visibility_timeout(url):
return url.startswith("redis://") or url.startswith("sqs://")


log = LoggingMixin().log
log = logging.getLogger(__name__)

broker_url = conf.get('celery', 'BROKER_URL')

Expand Down
4 changes: 2 additions & 2 deletions airflow/configuration.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
# under the License.

import copy
import logging
import os
import pathlib
import shlex
Expand All @@ -34,9 +35,8 @@
from zope.deprecation import deprecated

from airflow.exceptions import AirflowConfigException
from airflow.utils.log.logging_mixin import LoggingMixin

log = LoggingMixin().log
log = logging.getLogger(__name__)

# show Airflow's deprecation warnings
warnings.filterwarnings(
Expand Down
5 changes: 3 additions & 2 deletions airflow/contrib/auth/backends/github_enterprise_auth.py
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,8 @@
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
import logging

import flask_login
from flask import redirect, request, url_for
# Need to expose these downstream
Expand All @@ -25,10 +27,9 @@

from airflow import models
from airflow.configuration import AirflowConfigException, conf
from airflow.utils.log.logging_mixin import LoggingMixin
from airflow.utils.session import provide_session

log = LoggingMixin().log
log = logging.getLogger(__name__)


def get_config_param(param):
Expand Down
5 changes: 3 additions & 2 deletions airflow/contrib/auth/backends/google_auth.py
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,8 @@
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
import logging

import flask_login
from flask import redirect, request, url_for
# Need to expose these downstream
Expand All @@ -25,10 +27,9 @@

from airflow import models
from airflow.configuration import conf
from airflow.utils.log.logging_mixin import LoggingMixin
from airflow.utils.session import provide_session

log = LoggingMixin().log
log = logging.getLogger(__name__)


def get_config_param(param):
Expand Down
5 changes: 2 additions & 3 deletions airflow/contrib/auth/backends/ldap_auth.py
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.

import logging
import re
import ssl
import traceback
Expand All @@ -30,14 +30,13 @@

from airflow import models
from airflow.configuration import AirflowConfigException, conf
from airflow.utils.log.logging_mixin import LoggingMixin
from airflow.utils.session import provide_session

login_manager = flask_login.LoginManager()
login_manager.login_view = 'airflow.login' # Calls login() below
login_manager.login_message = None

log = LoggingMixin().log
log = logging.getLogger(__name__)


class AuthenticationError(Exception):
Expand Down
8 changes: 4 additions & 4 deletions airflow/contrib/auth/backends/password_auth.py
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
# under the License.
"""Password authentication backend"""
import base64
import logging
from functools import wraps

import flask_login
Expand All @@ -32,14 +33,13 @@
from wtforms.validators import InputRequired

from airflow import models
from airflow.utils.log.logging_mixin import LoggingMixin
from airflow.utils.session import create_session, provide_session

LOGIN_MANAGER = flask_login.LoginManager()
LOGIN_MANAGER.login_view = 'airflow.login' # Calls login() below
LOGIN_MANAGER.login_message = None

LOG = LoggingMixin().log
log = logging.getLogger(__name__)


CLIENT_AUTH = None
Expand Down Expand Up @@ -108,7 +108,7 @@ def is_superuser(self):
@provide_session
def load_user(userid, session=None):
"""Loads user from the database"""
LOG.debug("Loading user %s", userid)
log.debug("Loading user %s", userid)
if not userid or userid == 'None':
return None

Expand Down Expand Up @@ -140,7 +140,7 @@ def authenticate(session, username, password):
if not user.authenticate(password):
raise AuthenticationError()

LOG.info("User %s successfully authenticated", username)
log.info("User %s successfully authenticated", username)
return user


Expand Down
5 changes: 3 additions & 2 deletions airflow/contrib/utils/sendgrid.py
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
"""

import base64
import logging
import mimetypes
import os

Expand All @@ -30,7 +31,8 @@
)

from airflow.utils.email import get_email_address_list
from airflow.utils.log.logging_mixin import LoggingMixin

log = logging.getLogger(__name__)


def send_email(to, subject, html_content, files=None, cc=None,
Expand Down Expand Up @@ -114,7 +116,6 @@ def send_email(to, subject, html_content, files=None, cc=None,


def _post_sendgrid_mail(mail_data):
log = LoggingMixin().log
sendgrid_client = sendgrid.SendGridAPIClient(api_key=os.environ.get('SENDGRID_API_KEY'))
response = sendgrid_client.client.mail.send.post(request_body=mail_data)
# 2xx status code.
Expand Down
Loading

0 comments on commit cf14150

Please sign in to comment.
  翻译: